View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.regionserver;
21  
22  import java.io.IOException;
23  import java.io.StringWriter;
24  import java.lang.Thread.UncaughtExceptionHandler;
25  import java.lang.annotation.Retention;
26  import java.lang.annotation.RetentionPolicy;
27  import java.lang.management.ManagementFactory;
28  import java.lang.management.MemoryUsage;
29  import java.lang.reflect.Constructor;
30  import java.lang.reflect.Method;
31  import java.net.BindException;
32  import java.net.InetSocketAddress;
33  import java.util.ArrayList;
34  import java.util.Collection;
35  import java.util.Collections;
36  import java.util.Comparator;
37  import java.util.HashMap;
38  import java.util.HashSet;
39  import java.util.LinkedList;
40  import java.util.List;
41  import java.util.Map;
42  import java.util.Map.Entry;
43  import java.util.Random;
44  import java.util.Set;
45  import java.util.SortedMap;
46  import java.util.TreeMap;
47  import java.util.TreeSet;
48  import java.util.concurrent.ConcurrentHashMap;
49  import java.util.concurrent.ConcurrentSkipListMap;
50  import java.util.concurrent.atomic.AtomicBoolean;
51  import java.util.concurrent.atomic.AtomicInteger;
52  import java.util.concurrent.locks.ReentrantReadWriteLock;
53  
54  import javax.management.ObjectName;
55  
56  import org.apache.commons.lang.mutable.MutableDouble;
57  import org.apache.commons.logging.Log;
58  import org.apache.commons.logging.LogFactory;
59  import org.apache.hadoop.conf.Configuration;
60  import org.apache.hadoop.fs.FileSystem;
61  import org.apache.hadoop.fs.Path;
62  import org.apache.hadoop.hbase.Chore;
63  import org.apache.hadoop.hbase.ClockOutOfSyncException;
64  import org.apache.hadoop.hbase.DoNotRetryIOException;
65  import org.apache.hadoop.hbase.HBaseConfiguration;
66  import org.apache.hadoop.hbase.HConstants;
67  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
68  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
69  import org.apache.hadoop.hbase.HRegionInfo;
70  import org.apache.hadoop.hbase.HServerAddress;
71  import org.apache.hadoop.hbase.HServerInfo;
72  import org.apache.hadoop.hbase.HServerLoad;
73  import org.apache.hadoop.hbase.HTableDescriptor;
74  import org.apache.hadoop.hbase.HealthCheckChore;
75  import org.apache.hadoop.hbase.KeyValue;
76  import org.apache.hadoop.hbase.MasterAddressTracker;
77  import org.apache.hadoop.hbase.NotServingRegionException;
78  import org.apache.hadoop.hbase.RemoteExceptionHandler;
79  import org.apache.hadoop.hbase.ServerName;
80  import org.apache.hadoop.hbase.Stoppable;
81  import org.apache.hadoop.hbase.TableDescriptors;
82  import org.apache.hadoop.hbase.UnknownRowLockException;
83  import org.apache.hadoop.hbase.UnknownScannerException;
84  import org.apache.hadoop.hbase.YouAreDeadException;
85  import org.apache.hadoop.hbase.catalog.CatalogTracker;
86  import org.apache.hadoop.hbase.catalog.MetaEditor;
87  import org.apache.hadoop.hbase.catalog.MetaReader;
88  import org.apache.hadoop.hbase.catalog.RootLocationEditor;
89  import org.apache.hadoop.hbase.client.Action;
90  import org.apache.hadoop.hbase.client.Append;
91  import org.apache.hadoop.hbase.client.Delete;
92  import org.apache.hadoop.hbase.client.Get;
93  import org.apache.hadoop.hbase.client.HConnectionManager;
94  import org.apache.hadoop.hbase.client.Increment;
95  import org.apache.hadoop.hbase.client.MultiAction;
96  import org.apache.hadoop.hbase.client.MultiResponse;
97  import org.apache.hadoop.hbase.client.Mutation;
98  import org.apache.hadoop.hbase.client.Put;
99  import org.apache.hadoop.hbase.client.Result;
100 import org.apache.hadoop.hbase.client.Row;
101 import org.apache.hadoop.hbase.client.RowLock;
102 import org.apache.hadoop.hbase.client.RowMutations;
103 import org.apache.hadoop.hbase.client.Scan;
104 import org.apache.hadoop.hbase.client.UserProvider;
105 import org.apache.hadoop.hbase.client.coprocessor.Exec;
106 import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
107 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
108 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
109 import org.apache.hadoop.hbase.executor.ExecutorService;
110 import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
111 import org.apache.hadoop.hbase.filter.BinaryComparator;
112 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
113 import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
114 import org.apache.hadoop.hbase.fs.HFileSystem;
115 import org.apache.hadoop.hbase.io.hfile.BlockCache;
116 import org.apache.hadoop.hbase.io.hfile.BlockCacheColumnFamilySummary;
117 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
118 import org.apache.hadoop.hbase.io.hfile.CacheStats;
119 import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
120 import org.apache.hadoop.hbase.ipc.HBaseRPC;
121 import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
122 import org.apache.hadoop.hbase.ipc.HBaseRpcMetrics;
123 import org.apache.hadoop.hbase.ipc.HBaseServer;
124 import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
125 import org.apache.hadoop.hbase.ipc.HRegionInterface;
126 import org.apache.hadoop.hbase.ipc.Invocation;
127 import org.apache.hadoop.hbase.ipc.ProtocolSignature;
128 import org.apache.hadoop.hbase.ipc.RpcEngine;
129 import org.apache.hadoop.hbase.ipc.RpcServer;
130 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
131 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
132 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
133 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
134 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
135 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
136 import org.apache.hadoop.hbase.regionserver.handler.CloseRootHandler;
137 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
138 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
139 import org.apache.hadoop.hbase.regionserver.handler.OpenRootHandler;
140 import org.apache.hadoop.hbase.regionserver.metrics.RegionMetricsStorage;
141 import org.apache.hadoop.hbase.regionserver.metrics.RegionServerDynamicMetrics;
142 import org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
143 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
144 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.StoreMetricType;
145 import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
146 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
147 import org.apache.hadoop.hbase.regionserver.wal.HLog;
148 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
149 import org.apache.hadoop.hbase.security.User;
150 import org.apache.hadoop.hbase.util.Bytes;
151 import org.apache.hadoop.hbase.util.CompressionTest;
152 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
153 import org.apache.hadoop.hbase.util.FSTableDescriptors;
154 import org.apache.hadoop.hbase.util.FSUtils;
155 import org.apache.hadoop.hbase.util.InfoServer;
156 import org.apache.hadoop.hbase.util.Pair;
157 import org.apache.hadoop.hbase.util.Sleeper;
158 import org.apache.hadoop.hbase.util.Strings;
159 import org.apache.hadoop.hbase.util.Threads;
160 import org.apache.hadoop.hbase.util.VersionInfo;
161 import org.apache.hadoop.hbase.zookeeper.ClusterId;
162 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
163 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
164 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
165 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
166 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
167 import org.apache.hadoop.io.MapWritable;
168 import org.apache.hadoop.io.Writable;
169 import org.apache.hadoop.ipc.RemoteException;
170 import org.apache.hadoop.metrics.util.MBeanUtil;
171 import org.apache.hadoop.net.DNS;
172 import org.apache.hadoop.util.ReflectionUtils;
173 import org.apache.hadoop.util.StringUtils;
174 import org.apache.zookeeper.KeeperException;
175 import org.codehaus.jackson.map.ObjectMapper;
176 
177 import com.google.common.base.Function;
178 import com.google.common.collect.Lists;
179 
180 /**
181  * HRegionServer makes a set of HRegions available to clients. It checks in with
182  * the HMaster. There are many HRegionServers in a single HBase deployment.
183  */
184 public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
185     Runnable, RegionServerServices {
186 
187   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
188 
189   // Set when a report to the master comes back with a message asking us to
190   // shutdown. Also set by call to stop when debugging or running unit tests
191   // of HRegionServer in isolation.
192   protected volatile boolean stopped = false;
193 
194   // A state before we go into stopped state.  At this stage we're closing user
195   // space regions.
196   private boolean stopping = false;
197 
198   // Go down hard. Used if file system becomes unavailable and also in
199   // debugging and unit tests.
200   protected volatile boolean abortRequested;
201 
202   private volatile boolean killed = false;
203 
204   // If false, the file system has become unavailable
205   protected volatile boolean fsOk;
206 
207   protected final Configuration conf;
208 
209   protected final AtomicBoolean haveRootRegion = new AtomicBoolean(false);
210   private HFileSystem fs;
211   private boolean useHBaseChecksum; // verify hbase checksums?
212   private Path rootDir;
213   private final Random rand;
214 
215   //RegionName vs current action in progress
216   //true - if open region action in progress
217   //false - if close region action in progress
218   private final ConcurrentSkipListMap<byte[], Boolean> regionsInTransitionInRS =
219       new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
220 
221   /**
222    * Map of regions currently being served by this region server. Key is the
223    * encoded region name.  All access should be synchronized.
224    */
225   protected final Map<String, HRegion> onlineRegions =
226     new ConcurrentHashMap<String, HRegion>();
227 
228   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
229 
230   final int numRetries;
231   protected final int threadWakeFrequency;
232   private final int msgInterval;
233 
234   protected final int numRegionsToReport;
235 
236   private final long maxScannerResultSize;
237 
238   // Remote HMaster
239   private HMasterRegionInterface hbaseMaster;
240 
241   // RPC Engine for master connection
242   private RpcEngine rpcEngine;
243 
244   // Server to handle client requests. Default access so can be accessed by
245   // unit tests.
246   RpcServer rpcServer;
247 
248   // Server to handle client requests.
249   private HBaseServer server;  
250 
251   private final InetSocketAddress isa;
252   private UncaughtExceptionHandler uncaughtExceptionHandler;
253 
254   // Leases
255   private Leases leases;
256 
257   // Request counter.
258   // Do we need this?  Can't we just sum region counters?  St.Ack 20110412
259   private AtomicInteger requestCount = new AtomicInteger();
260 
261   // Info server. Default access so can be used by unit tests. REGIONSERVER
262   // is name of the webapp and the attribute name used stuffing this instance
263   // into web context.
264   InfoServer infoServer;
265 
266   /** region server process name */
267   public static final String REGIONSERVER = "regionserver";
268   
269   /** region server configuration name */
270   public static final String REGIONSERVER_CONF = "regionserver_conf";
271 
272   /*
273    * Space is reserved in HRS constructor and then released when aborting to
274    * recover from an OOME. See HBASE-706. TODO: Make this percentage of the heap
275    * or a minimum.
276    */
277   private final LinkedList<byte[]> reservedSpace = new LinkedList<byte[]>();
278 
279   private RegionServerMetrics metrics;
280 
281   private RegionServerDynamicMetrics dynamicMetrics;
282 
283   // Compactions
284   public CompactSplitThread compactSplitThread;
285 
286   // Cache flushing
287   MemStoreFlusher cacheFlusher;
288 
289   /*
290    * Check for compactions requests.
291    */
292   Chore compactionChecker;
293 
294   /*
295    * Check for flushes
296    */
297   Chore periodicFlusher;
298 
299   // HLog and HLog roller. log is protected rather than private to avoid
300   // eclipse warning when accessed by inner classes
301   protected volatile HLog hlog;
302   // The meta updates are written to a different hlog. If this
303   // regionserver holds meta regions, then this field will be non-null.
304   protected volatile HLog hlogForMeta;
305 
306   LogRoller hlogRoller;
307   LogRoller metaHLogRoller;
308 
309   private final boolean separateHLogForMeta;
310 
311   // flag set after we're done setting up server threads (used for testing)
312   protected volatile boolean isOnline;
313 
314   final Map<String, RegionScanner> scanners =
315     new ConcurrentHashMap<String, RegionScanner>();
316 
317   // zookeeper connection and watcher
318   private ZooKeeperWatcher zooKeeper;
319 
320   // master address manager and watcher
321   private MasterAddressTracker masterAddressManager;
322 
323   // catalog tracker
324   private CatalogTracker catalogTracker;
325 
326   // Cluster Status Tracker
327   private ClusterStatusTracker clusterStatusTracker;
328 
329   // Log Splitting Worker
330   private SplitLogWorker splitLogWorker;
331 
332   // A sleeper that sleeps for msgInterval.
333   private final Sleeper sleeper;
334 
335   private final int rpcTimeout;
336 
337   // Instance of the hbase executor service.
338   private ExecutorService service;
339 
340   // Replication services. If no replication, this handler will be null.
341   private ReplicationSourceService replicationSourceHandler;
342   private ReplicationSinkService replicationSinkHandler;
343 
344   private final RegionServerAccounting regionServerAccounting;
345 
346   // Cache configuration and block cache reference
347   private final CacheConfig cacheConfig;
348 
349   // reference to the Thrift Server.
350   volatile private HRegionThriftServer thriftServer;
351 
352   /**
353    * The server name the Master sees us as.  Its made from the hostname the
354    * master passes us, port, and server startcode. Gets set after registration
355    * against  Master.  The hostname can differ from the hostname in {@link #isa}
356    * but usually doesn't if both servers resolve .
357    */
358   private ServerName serverNameFromMasterPOV;
359 
360   // Port we put up the webui on.
361   private int webuiport = -1;
362 
363   /**
364    * This servers startcode.
365    */
366   private final long startcode;
367 
368   /**
369    * Go here to get table descriptors.
370    */
371   private TableDescriptors tableDescriptors;
372 
373   /*
374    * Strings to be used in forming the exception message for
375    * RegionsAlreadyInTransitionException.
376    */
377   private static final String OPEN = "OPEN";
378   private static final String CLOSE = "CLOSE";
379 
380   /**
381    * MX Bean for RegionServerInfo
382    */
383   private ObjectName mxBean = null;
384 
385   /**
386    * ClusterId
387    */
388   private ClusterId clusterId = null;
389 
390   private RegionServerCoprocessorHost rsHost;
391 
392   /** The health check chore. */
393   private HealthCheckChore healthCheckChore;
394 
395   /**
396    * Starts a HRegionServer at the default location
397    *
398    * @param conf
399    * @throws IOException
400    * @throws InterruptedException
401    */
402   public HRegionServer(Configuration conf)
403   throws IOException, InterruptedException {
404     this.fsOk = true;
405     this.conf = conf;
406     // Set how many times to retry talking to another server over HConnection.
407     HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
408     this.isOnline = false;
409     checkCodecs(this.conf);
410 
411     // do we use checksum verfication in the hbase? If hbase checksum verification
412     // is enabled, then we automatically switch off hdfs checksum verification.
413     this.useHBaseChecksum = conf.getBoolean(
414       HConstants.HBASE_CHECKSUM_VERIFICATION, false);
415 
416     // Config'ed params
417     this.separateHLogForMeta = conf.getBoolean(HLog.SEPARATE_HLOG_FOR_META, false);
418     this.numRetries = conf.getInt("hbase.client.retries.number", 10);
419     this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY,
420       10 * 1000);
421     this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
422 
423     this.sleeper = new Sleeper(this.msgInterval, this);
424 
425     this.maxScannerResultSize = conf.getLong(
426       HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
427       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
428 
429     this.numRegionsToReport = conf.getInt(
430       "hbase.regionserver.numregionstoreport", 10);
431 
432     this.rpcTimeout = conf.getInt(
433       HConstants.HBASE_RPC_SHORTOPERATION_TIMEOUT_KEY,
434       HConstants.DEFAULT_HBASE_RPC_SHORTOPERATION_TIMEOUT);
435 
436     this.abortRequested = false;
437     this.stopped = false;
438 
439     // Server to handle client requests.
440     String hostname = conf.get("hbase.regionserver.ipc.address",
441       Strings.domainNamePointerToHostName(DNS.getDefaultHost(
442         conf.get("hbase.regionserver.dns.interface", "default"),
443         conf.get("hbase.regionserver.dns.nameserver", "default"))));
444     int port = conf.getInt(HConstants.REGIONSERVER_PORT,
445       HConstants.DEFAULT_REGIONSERVER_PORT);
446     // Creation of a HSA will force a resolve.
447     InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
448     if (initialIsa.getAddress() == null) {
449       throw new IllegalArgumentException("Failed resolve of " + initialIsa);
450     }
451 
452     this.rand = new Random(initialIsa.hashCode());
453     this.rpcServer = HBaseRPC.getServer(this,
454       new Class<?>[]{HRegionInterface.class, HBaseRPCErrorHandler.class,
455         OnlineRegions.class},
456         initialIsa.getHostName(), // BindAddress is IP we got for this server.
457         initialIsa.getPort(),
458         conf.getInt("hbase.regionserver.handler.count", 10),
459         conf.getInt("hbase.regionserver.metahandler.count", 10),
460         conf.getBoolean("hbase.rpc.verbose", false),
461         conf, HConstants.QOS_THRESHOLD);
462     if (rpcServer instanceof HBaseServer) server = (HBaseServer) rpcServer;
463     // Set our address.
464     this.isa = this.rpcServer.getListenerAddress();
465 
466     this.rpcServer.setErrorHandler(this);
467     this.rpcServer.setQosFunction(new QosFunction());
468     this.startcode = System.currentTimeMillis();
469 
470     conf.set("hbase.regionserver.rpc.client.socket.bind.address", this.isa.getHostName());
471 
472     // login the zookeeper client principal (if using security)
473     ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
474       "hbase.zookeeper.client.kerberos.principal", this.isa.getHostName());
475 
476     // login the server principal (if using secure Hadoop)
477     UserProvider provider = UserProvider.instantiate(conf);
478     provider.login("hbase.regionserver.keytab.file",
479       "hbase.regionserver.kerberos.principal", this.isa.getHostName());
480     regionServerAccounting = new RegionServerAccounting();
481     cacheConfig = new CacheConfig(conf);
482     uncaughtExceptionHandler = new UncaughtExceptionHandler() {
483       public void uncaughtException(Thread t, Throwable e) {
484         abort("Uncaught exception in service thread " + t.getName(), e);
485       }
486     };
487   }
488 
489   /** Handle all the snapshot requests to this server */
490   RegionServerSnapshotManager snapshotManager;
491 
492   /**
493    * Run test on configured codecs to make sure supporting libs are in place.
494    * @param c
495    * @throws IOException
496    */
497   private static void checkCodecs(final Configuration c) throws IOException {
498     // check to see if the codec list is available:
499     String [] codecs = c.getStrings("hbase.regionserver.codecs", (String[])null);
500     if (codecs == null) return;
501     for (String codec : codecs) {
502       if (!CompressionTest.testCompression(codec)) {
503         throw new IOException("Compression codec " + codec +
504           " not supported, aborting RS construction");
505       }
506     }
507   }
508 
509 
510   @Retention(RetentionPolicy.RUNTIME)
511   private @interface QosPriority {
512     int priority() default 0;
513   }
514 
515   /**
516    * Utility used ensuring higher quality of service for priority rpcs; e.g.
517    * rpcs to .META. and -ROOT-, etc.
518    */
519   class QosFunction implements Function<Writable,Integer> {
520     private final Map<String, Integer> annotatedQos;
521 
522     public QosFunction() {
523       Map<String, Integer> qosMap = new HashMap<String, Integer>();
524       for (Method m : HRegionServer.class.getMethods()) {
525         QosPriority p = m.getAnnotation(QosPriority.class);
526         if (p != null) {
527           qosMap.put(m.getName(), p.priority());
528         }
529       }
530 
531       annotatedQos = qosMap;
532     }
533 
534     public boolean isMetaTable(byte[] regionName) {
535       HRegion region;
536       try {
537         region = getRegion(regionName);
538       } catch (NotServingRegionException ignored) {
539         return false;
540       }
541       return region.getRegionInfo().isMetaTable();
542     }
543 
544     @Override
545     public Integer apply(Writable from) {
546       if (!(from instanceof Invocation)) return HConstants.NORMAL_QOS;
547 
548       Invocation inv = (Invocation) from;
549       String methodName = inv.getMethodName();
550 
551       Integer priorityByAnnotation = annotatedQos.get(methodName);
552       if (priorityByAnnotation != null) {
553         return priorityByAnnotation;
554       }
555 
556       // scanner methods...
557       if (methodName.equals("next") || methodName.equals("close")) {
558         // translate!
559         Long scannerId;
560         try {
561           scannerId = (Long) inv.getParameters()[0];
562         } catch (ClassCastException ignored) {
563           // LOG.debug("Low priority: " + from);
564           return HConstants.NORMAL_QOS;
565         }
566         String scannerIdString = Long.toString(scannerId);
567         RegionScanner scanner = scanners.get(scannerIdString);
568         if (scanner != null && scanner.getRegionInfo().isMetaTable()) {
569           // LOG.debug("High priority scanner request: " + scannerId);
570           return HConstants.HIGH_QOS;
571         }
572       } else if (inv.getParameterClasses().length == 0) {
573        // Just let it through.  This is getOnlineRegions, etc.
574       } else if (inv.getParameterClasses()[0] == byte[].class) {
575         // first arg is byte array, so assume this is a regionname:
576         if (isMetaTable((byte[]) inv.getParameters()[0])) {
577           // LOG.debug("High priority with method: " + methodName +
578           // " and region: "
579           // + Bytes.toString((byte[]) inv.getParameters()[0]));
580           return HConstants.HIGH_QOS;
581         }
582       } else if (inv.getParameterClasses()[0] == MultiAction.class) {
583         MultiAction<?> ma = (MultiAction<?>) inv.getParameters()[0];
584         Set<byte[]> regions = ma.getRegions();
585         // ok this sucks, but if any single of the actions touches a meta, the
586         // whole
587         // thing gets pingged high priority. This is a dangerous hack because
588         // people
589         // can get their multi action tagged high QOS by tossing a Get(.META.)
590         // AND this
591         // regionserver hosts META/-ROOT-
592         for (byte[] region : regions) {
593           if (isMetaTable(region)) {
594             // LOG.debug("High priority multi with region: " +
595             // Bytes.toString(region));
596             return HConstants.HIGH_QOS; // short circuit for the win.
597           }
598         }
599       }
600       // LOG.debug("Low priority: " + from.toString());
601       return HConstants.NORMAL_QOS;
602     }
603   }
604 
605   /**
606    * All initialization needed before we go register with Master.
607    *
608    * @throws IOException
609    * @throws InterruptedException
610    */
611   private void preRegistrationInitialization(){
612     try {
613       initializeZooKeeper();
614 
615       clusterId = new ClusterId(zooKeeper, this);
616       if(clusterId.hasId()) {
617         conf.set(HConstants.CLUSTER_ID, clusterId.getId());
618       }
619 
620       initializeThreads();
621       int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4);
622       for (int i = 0; i < nbBlocks; i++) {
623         reservedSpace.add(new byte[HConstants.DEFAULT_SIZE_RESERVATION_BLOCK]);
624       }
625 
626       this.rpcEngine = HBaseRPC.getProtocolEngine(conf);
627     } catch (Throwable t) {
628       // Call stop if error or process will stick around for ever since server
629       // puts up non-daemon threads.
630       this.rpcServer.stop();
631       abort("Initialization of RS failed.  Hence aborting RS.", t);
632     }
633   }
634 
635   /**
636    * Bring up connection to zk ensemble and then wait until a master for this
637    * cluster and then after that, wait until cluster 'up' flag has been set.
638    * This is the order in which master does things.
639    * Finally put up a catalog tracker.
640    * @throws IOException
641    * @throws InterruptedException
642    */
643   private void initializeZooKeeper() throws IOException, InterruptedException {
644     // Open connection to zookeeper and set primary watcher
645     this.zooKeeper = new ZooKeeperWatcher(conf, REGIONSERVER + ":" +
646       this.isa.getPort(), this);
647 
648     // Create the master address manager, register with zk, and start it.  Then
649     // block until a master is available.  No point in starting up if no master
650     // running.
651     this.masterAddressManager = new MasterAddressTracker(this.zooKeeper, this);
652     this.masterAddressManager.start();
653     blockAndCheckIfStopped(this.masterAddressManager);
654 
655     // Wait on cluster being up.  Master will set this flag up in zookeeper
656     // when ready.
657     this.clusterStatusTracker = new ClusterStatusTracker(this.zooKeeper, this);
658     this.clusterStatusTracker.start();
659     blockAndCheckIfStopped(this.clusterStatusTracker);
660 
661     // Create the catalog tracker and start it;
662     this.catalogTracker = new CatalogTracker(this.zooKeeper, this.conf, this);
663     catalogTracker.start();
664 
665     // watch for snapshots
666     try {
667       this.snapshotManager = new RegionServerSnapshotManager(this);
668     } catch (KeeperException e) {
669       this.abort("Failed to reach zk cluster when creating snapshot handler.");
670     }
671   }
672 
673   /**
674    * Utilty method to wait indefinitely on a znode availability while checking
675    * if the region server is shut down
676    * @param tracker znode tracker to use
677    * @throws IOException any IO exception, plus if the RS is stopped
678    * @throws InterruptedException
679    */
680   private void blockAndCheckIfStopped(ZooKeeperNodeTracker tracker)
681       throws IOException, InterruptedException {
682     while (tracker.blockUntilAvailable(this.msgInterval, false) == null) {
683       if (this.stopped) {
684         throw new IOException("Received the shutdown message while waiting.");
685       }
686     }
687   }
688 
689   /**
690    * @return False if cluster shutdown in progress
691    */
692   private boolean isClusterUp() {
693     return this.clusterStatusTracker.isClusterUp();
694   }
695 
696   private void initializeThreads() throws IOException {
697     // Cache flushing thread.
698     this.cacheFlusher = new MemStoreFlusher(conf, this);
699 
700     // Compaction thread
701     this.compactSplitThread = new CompactSplitThread(this);
702 
703     // Background thread to check for compactions; needed if region
704     // has not gotten updates in a while. Make it run at a lesser frequency.
705     int multiplier = this.conf.getInt(HConstants.THREAD_WAKE_FREQUENCY +
706       ".multiplier", 1000);
707     this.compactionChecker = new CompactionChecker(this,
708       this.threadWakeFrequency * multiplier, this);
709 
710     this.periodicFlusher = new PeriodicMemstoreFlusher(this.threadWakeFrequency, this);
711 
712     // Health checker thread.
713     int sleepTime = this.conf.getInt(HConstants.HEALTH_CHORE_WAKE_FREQ,
714       HConstants.DEFAULT_THREAD_WAKE_FREQUENCY);
715     if (isHealthCheckerConfigured()) {
716       healthCheckChore = new HealthCheckChore(sleepTime, this, getConfiguration());
717     }
718 
719     this.leases = new Leases((int) conf.getLong(
720         HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
721         HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD),
722         this.threadWakeFrequency);
723 
724     // Create the thread for the ThriftServer.
725     if (conf.getBoolean("hbase.regionserver.export.thrift", false)) {
726       thriftServer = new HRegionThriftServer(this, conf);
727       thriftServer.start();
728       LOG.info("Started Thrift API from Region Server.");
729     }
730   }
731 
732   /**
733    * The HRegionServer sticks in this loop until closed.
734    */
735   @SuppressWarnings("deprecation")
736   public void run() {
737     try {
738       // Do pre-registration initializations; zookeeper, lease threads, etc.
739       preRegistrationInitialization();
740     } catch (Throwable e) {
741       abort("Fatal exception during initialization", e);
742     }
743 
744     try {
745       // Set our ephemeral znode up in zookeeper now we have a name.
746       createMyEphemeralNode();
747    
748       // Try and register with the Master; tell it we are here.  Break if
749       // server is stopped or the clusterup flag is down or hdfs went wacky.
750       while (keepLooping()) {
751         MapWritable w = reportForDuty();
752         if (w == null) {
753           LOG.warn("reportForDuty failed; sleeping and then retrying.");
754           this.sleeper.sleep();
755         } else {
756           handleReportForDutyResponse(w);
757           break;
758         }
759       }
760       registerMBean();
761 
762       // start the snapshot handler, since the server is ready to run
763       this.snapshotManager.start();
764 
765       // We registered with the Master.  Go into run mode.
766       long lastMsg = 0;
767       long oldRequestCount = -1;
768       // The main run loop.
769       while (!this.stopped && isHealthy()) {
770         if (!isClusterUp()) {
771           if (isOnlineRegionsEmpty()) {
772             stop("Exiting; cluster shutdown set and not carrying any regions");
773           } else if (!this.stopping) {
774             this.stopping = true;
775             LOG.info("Closing user regions");
776             closeUserRegions(this.abortRequested);
777           } else if (this.stopping) {
778             boolean allUserRegionsOffline = areAllUserRegionsOffline();
779             if (allUserRegionsOffline) {
780               // Set stopped if no requests since last time we went around the loop.
781               // The remaining meta regions will be closed on our way out.
782               if (oldRequestCount == this.requestCount.get()) {
783                 stop("Stopped; only catalog regions remaining online");
784                 break;
785               }
786               oldRequestCount = this.requestCount.get();
787             } else {
788               // Make sure all regions have been closed -- some regions may
789               // have not got it because we were splitting at the time of
790               // the call to closeUserRegions.
791               closeUserRegions(this.abortRequested);
792             }
793             LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
794           }
795         }
796         long now = System.currentTimeMillis();
797         if ((now - lastMsg) >= msgInterval) {
798           doMetrics();
799           tryRegionServerReport();
800           lastMsg = System.currentTimeMillis();
801         }
802         if (!this.stopped) this.sleeper.sleep();
803       } // for
804     } catch (Throwable t) {
805       if (!checkOOME(t)) {
806         abort("Unhandled exception: " + t.getMessage(), t);
807       }
808     }
809     // Run shutdown.
810     if (mxBean != null) {
811       MBeanUtil.unregisterMBean(mxBean);
812       mxBean = null;
813     }
814     if (this.thriftServer != null) this.thriftServer.shutdown();
815     this.leases.closeAfterLeasesExpire();
816     this.rpcServer.stop();
817     if (this.splitLogWorker != null) {
818       splitLogWorker.stop();
819     }
820     if (this.infoServer != null) {
821       LOG.info("Stopping infoServer");
822       try {
823         this.infoServer.stop();
824       } catch (Exception e) {
825         e.printStackTrace();
826       }
827     }
828     // Send cache a shutdown.
829     if (cacheConfig.isBlockCacheEnabled()) {
830       cacheConfig.getBlockCache().shutdown();
831     }
832 
833     // Send interrupts to wake up threads if sleeping so they notice shutdown.
834     // TODO: Should we check they are alive? If OOME could have exited already
835     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
836     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
837     if (this.hlogRoller != null) this.hlogRoller.interruptIfNecessary();
838     if (this.metaHLogRoller != null) this.metaHLogRoller.interruptIfNecessary();
839     if (this.compactionChecker != null)
840       this.compactionChecker.interrupt();
841     if (this.healthCheckChore != null) {
842       this.healthCheckChore.interrupt();
843     }
844 
845     // Stop the snapshot handler, forcefully killing all running tasks
846     try {
847       if (snapshotManager != null) snapshotManager.stop(this.abortRequested || this.killed);
848     } catch (IOException e) {
849       LOG.warn("Failed to close snapshot handler cleanly", e);
850     }
851 
852     if (this.killed) {
853       // Just skip out w/o closing regions.  Used when testing.
854     } else if (abortRequested) {
855       if (this.fsOk) {
856         closeUserRegions(abortRequested); // Don't leave any open file handles
857       }
858       LOG.info("aborting server " + this.serverNameFromMasterPOV);
859     } else {
860       closeUserRegions(abortRequested);
861       closeAllScanners();
862       LOG.info("stopping server " + this.serverNameFromMasterPOV);
863     }
864     // Interrupt catalog tracker here in case any regions being opened out in
865     // handlers are stuck waiting on meta or root.
866     if (this.catalogTracker != null) this.catalogTracker.stop();
867 
868     // Closing the compactSplit thread before closing meta regions
869     if (!this.killed && containsMetaTableRegions()) {
870       if (!abortRequested || this.fsOk) {
871         if (this.compactSplitThread != null) {
872           this.compactSplitThread.join();
873           this.compactSplitThread = null;
874         }
875         closeMetaTableRegions(abortRequested);
876       }
877     }
878 
879     if (!this.killed && this.fsOk) {
880       waitOnAllRegionsToClose(abortRequested);
881       LOG.info("stopping server " + this.serverNameFromMasterPOV +
882         "; all regions closed.");
883     }
884 
885     //fsOk flag may be changed when closing regions throws exception.
886     if (!this.killed && this.fsOk) {
887       closeWAL(abortRequested ? false : true);
888     }
889 
890     // Make sure the proxy is down.
891     this.hbaseMaster = null;
892     this.rpcEngine.close();
893     this.leases.close();
894 
895     if (!killed) {
896       join();
897     }
898 
899     try {
900       deleteMyEphemeralNode();
901     } catch (KeeperException e) {
902       LOG.warn("Failed deleting my ephemeral node", e);
903     }
904     this.zooKeeper.close();
905     LOG.info("stopping server " + this.serverNameFromMasterPOV +
906       "; zookeeper connection closed.");
907 
908     LOG.info(Thread.currentThread().getName() + " exiting");
909   }
910 
911   private boolean containsMetaTableRegions() {
912     return onlineRegions.containsKey(HRegionInfo.ROOT_REGIONINFO.getEncodedName())
913         || onlineRegions.containsKey(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
914   }
915 
916   private boolean areAllUserRegionsOffline() {
917     if (getNumberOfOnlineRegions() > 2) return false;
918     boolean allUserRegionsOffline = true;
919     for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
920       if (!e.getValue().getRegionInfo().isMetaTable()) {
921         allUserRegionsOffline = false;
922         break;
923       }
924     }
925     return allUserRegionsOffline;
926   }
927 
928   void tryRegionServerReport()
929   throws IOException {
930     HServerLoad hsl = buildServerLoad();
931     // Why we do this?
932     this.requestCount.set(0);
933     try {
934       this.hbaseMaster.regionServerReport(this.serverNameFromMasterPOV.getVersionedBytes(), hsl);
935     } catch (IOException ioe) {
936       if (ioe instanceof RemoteException) {
937         ioe = ((RemoteException)ioe).unwrapRemoteException();
938       }
939       if (ioe instanceof YouAreDeadException) {
940         // This will be caught and handled as a fatal error in run()
941         throw ioe;
942       }
943       // Couldn't connect to the master, get location from zk and reconnect
944       // Method blocks until new master is found or we are stopped
945       getMaster();
946     }
947   }
948 
949   HServerLoad buildServerLoad() {
950     Collection<HRegion> regions = getOnlineRegionsLocalContext();
951     TreeMap<byte [], HServerLoad.RegionLoad> regionLoads =
952       new TreeMap<byte [], HServerLoad.RegionLoad>(Bytes.BYTES_COMPARATOR);
953     for (HRegion region: regions) {
954       regionLoads.put(region.getRegionName(), createRegionLoad(region));
955     }
956     MemoryUsage memory =
957       ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
958     return new HServerLoad(requestCount.get(),(int)metrics.getRequests(),
959       (int)(memory.getUsed() / 1024 / 1024),
960       (int) (memory.getMax() / 1024 / 1024), regionLoads,
961       this.hlog.getCoprocessorHost().getCoprocessors());
962   }
963 
964   String getOnlineRegionsAsPrintableString() {
965     StringBuilder sb = new StringBuilder();
966     for (HRegion r: this.onlineRegions.values()) {
967       if (sb.length() > 0) sb.append(", ");
968       sb.append(r.getRegionInfo().getEncodedName());
969     }
970     return sb.toString();
971   }
972 
973   /**
974    * Wait on regions close.
975    */
976   private void waitOnAllRegionsToClose(final boolean abort) {
977     // Wait till all regions are closed before going out.
978     int lastCount = -1;
979     long previousLogTime = 0;
980     Set<String> closedRegions = new HashSet<String>();
981     while (!isOnlineRegionsEmpty()) {
982       int count = getNumberOfOnlineRegions();
983       // Only print a message if the count of regions has changed.
984       if (count != lastCount) {
985         // Log every second at most
986         if (System.currentTimeMillis() > (previousLogTime + 1000)) {
987           previousLogTime = System.currentTimeMillis();
988           lastCount = count;
989           LOG.info("Waiting on " + count + " regions to close");
990           // Only print out regions still closing if a small number else will
991           // swamp the log.
992           if (count < 10 && LOG.isDebugEnabled()) {
993             LOG.debug(this.onlineRegions);
994           }
995         }
996       }
997       // Ensure all user regions have been sent a close. Use this to
998       // protect against the case where an open comes in after we start the
999       // iterator of onlineRegions to close all user regions.
1000       for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
1001         HRegionInfo hri = e.getValue().getRegionInfo();
1002         if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())
1003             && !closedRegions.contains(hri.getEncodedName())) {
1004           closedRegions.add(hri.getEncodedName());
1005           // Don't update zk with this close transition; pass false.
1006           closeRegion(hri, abort, false);
1007         }
1008       }
1009       // No regions in RIT, we could stop waiting now.
1010       if (this.regionsInTransitionInRS.isEmpty()) {
1011         if (!isOnlineRegionsEmpty()) {
1012           LOG.info("We were exiting though online regions are not empty, because some regions failed closing");
1013         }
1014         break;
1015       }
1016       Threads.sleep(200);
1017     }
1018   }
1019 
1020   private void closeWAL(final boolean delete) {
1021     if (this.hlogForMeta != null) {
1022       // All hlogs (meta and non-meta) are in the same directory. Don't call
1023       // closeAndDelete here since that would delete all hlogs not just the
1024       // meta ones. We will just 'close' the hlog for meta here, and leave
1025       // the directory cleanup to the follow-on closeAndDelete call.
1026       try { //Part of the patch from HBASE-7982 to do with exception handling 
1027         this.hlogForMeta.close();
1028       } catch (Throwable e) {
1029         LOG.error("Metalog close and delete failed", RemoteExceptionHandler.checkThrowable(e));
1030       }
1031     }
1032     if (this.hlog != null) {
1033       try {
1034         if (delete) {
1035           hlog.closeAndDelete();
1036         } else {
1037           hlog.close();
1038         }
1039       } catch (Throwable e) {
1040         LOG.error("Close and delete failed", RemoteExceptionHandler.checkThrowable(e));
1041       }
1042     }
1043   }
1044 
1045   private void closeAllScanners() {
1046     // Close any outstanding scanners. Means they'll get an UnknownScanner
1047     // exception next time they come in.
1048     for (Map.Entry<String, RegionScanner> e : this.scanners.entrySet()) {
1049       try {
1050         e.getValue().close();
1051       } catch (IOException ioe) {
1052         LOG.warn("Closing scanner " + e.getKey(), ioe);
1053       }
1054     }
1055   }
1056 
1057   /*
1058    * Run init. Sets up hlog and starts up all server threads.
1059    *
1060    * @param c Extra configuration.
1061    */
1062   protected void handleReportForDutyResponse(final MapWritable c)
1063   throws IOException {
1064     try {
1065       for (Map.Entry<Writable, Writable> e :c.entrySet()) {
1066         String key = e.getKey().toString();
1067         // The hostname the master sees us as.
1068         if (key.equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
1069           String hostnameFromMasterPOV = e.getValue().toString();
1070           this.serverNameFromMasterPOV = new ServerName(hostnameFromMasterPOV,
1071             this.isa.getPort(), this.startcode);
1072           LOG.info("Master passed us hostname to use. Was=" +
1073             this.isa.getHostName() + ", Now=" +
1074             this.serverNameFromMasterPOV.getHostname());
1075           continue;
1076         }
1077         String value = e.getValue().toString();
1078         if (LOG.isDebugEnabled()) {
1079           LOG.debug("Config from master: " + key + "=" + value);
1080         }
1081         this.conf.set(key, value);
1082       }
1083 
1084       // hack! Maps DFSClient => RegionServer for logs.  HDFS made this
1085       // config param for task trackers, but we can piggyback off of it.
1086       if (this.conf.get("mapred.task.id") == null) {
1087         this.conf.set("mapred.task.id", "hb_rs_" +
1088           this.serverNameFromMasterPOV.toString());
1089       }
1090 
1091       // Master sent us hbase.rootdir to use. Should be fully qualified
1092       // path with file system specification included. Set 'fs.defaultFS'
1093       // to match the filesystem on hbase.rootdir else underlying hadoop hdfs
1094       // accessors will be going against wrong filesystem (unless all is set
1095       // to defaults).
1096       this.conf.set("fs.defaultFS", this.conf.get("hbase.rootdir"));
1097       // Get fs instance used by this RS
1098       this.fs = new HFileSystem(this.conf, this.useHBaseChecksum);
1099       this.rootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
1100       this.tableDescriptors = new FSTableDescriptors(this.fs, this.rootDir, true);
1101       this.hlog = setupWALAndReplication();
1102       // Init in here rather than in constructor after thread name has been set
1103       this.metrics = new RegionServerMetrics();
1104       this.dynamicMetrics = RegionServerDynamicMetrics.newInstance(this);
1105       this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
1106       startServiceThreads();
1107       LOG.info("Serving as " + this.serverNameFromMasterPOV +
1108         ", RPC listening on " + this.isa +
1109         ", sessionid=0x" +
1110         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
1111       isOnline = true;
1112     } catch (Throwable e) {
1113       LOG.warn("Exception in region server : ", e);
1114       this.isOnline = false;
1115       stop("Failed initialization");
1116       throw convertThrowableToIOE(cleanup(e, "Failed init"),
1117           "Region server startup failed");
1118     } finally {
1119       sleeper.skipSleepCycle();
1120     }
1121   }
1122 
1123   private String getMyEphemeralNodePath() {
1124     return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString());
1125   }
1126 
1127   private void createMyEphemeralNode() throws KeeperException {
1128     ZKUtil.createEphemeralNodeAndWatch(this.zooKeeper, getMyEphemeralNodePath(),
1129       HConstants.EMPTY_BYTE_ARRAY);
1130   }
1131 
1132   private void deleteMyEphemeralNode() throws KeeperException {
1133     ZKUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath());
1134   }
1135 
1136   public RegionServerAccounting getRegionServerAccounting() {
1137     return regionServerAccounting;
1138   }
1139 
1140   /*
1141    * @param r Region to get RegionLoad for.
1142    *
1143    * @return RegionLoad instance.
1144    *
1145    * @throws IOException
1146    */
1147   private HServerLoad.RegionLoad createRegionLoad(final HRegion r) {
1148     byte[] name = r.getRegionName();
1149     int stores = 0;
1150     int storefiles = 0;
1151     int storeUncompressedSizeMB = 0;
1152     int storefileSizeMB = 0;
1153     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
1154     int storefileIndexSizeMB = 0;
1155     int rootIndexSizeKB = 0;
1156     int totalStaticIndexSizeKB = 0;
1157     int totalStaticBloomSizeKB = 0;
1158     long totalCompactingKVs = 0;
1159     long currentCompactedKVs = 0;
1160     synchronized (r.stores) {
1161       stores += r.stores.size();
1162       for (Store store : r.stores.values()) {
1163         storefiles += store.getStorefilesCount();
1164         storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed()
1165             / 1024 / 1024);
1166         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
1167         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
1168         CompactionProgress progress = store.getCompactionProgress();
1169         if (progress != null) {
1170           totalCompactingKVs += progress.totalCompactingKVs;
1171           currentCompactedKVs += progress.currentCompactedKVs;
1172         }
1173 
1174         rootIndexSizeKB +=
1175             (int) (store.getStorefilesIndexSize() / 1024);
1176 
1177         totalStaticIndexSizeKB +=
1178           (int) (store.getTotalStaticIndexSize() / 1024);
1179 
1180         totalStaticBloomSizeKB +=
1181           (int) (store.getTotalStaticBloomSize() / 1024);
1182       }
1183     }
1184     return new HServerLoad.RegionLoad(name, stores, storefiles,
1185         storeUncompressedSizeMB,
1186         storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB, rootIndexSizeKB,
1187         totalStaticIndexSizeKB, totalStaticBloomSizeKB,
1188         (int) r.readRequestsCount.get(), (int) r.writeRequestsCount.get(),
1189         totalCompactingKVs, currentCompactedKVs);
1190   }
1191 
1192   /**
1193    * @param encodedRegionName
1194    * @return An instance of RegionLoad.
1195    */
1196   public HServerLoad.RegionLoad createRegionLoad(final String encodedRegionName) {
1197     HRegion r = null;
1198     r = this.onlineRegions.get(encodedRegionName);
1199     return r != null ? createRegionLoad(r) : null;
1200   }
1201 
1202   /*
1203    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
1204    * IOE if it isn't already.
1205    *
1206    * @param t Throwable
1207    *
1208    * @return Throwable converted to an IOE; methods can only let out IOEs.
1209    */
1210   private Throwable cleanup(final Throwable t) {
1211     return cleanup(t, null);
1212   }
1213 
1214   /*
1215    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
1216    * IOE if it isn't already.
1217    *
1218    * @param t Throwable
1219    *
1220    * @param msg Message to log in error. Can be null.
1221    *
1222    * @return Throwable converted to an IOE; methods can only let out IOEs.
1223    */
1224   private Throwable cleanup(final Throwable t, final String msg) {
1225     // Don't log as error if NSRE; NSRE is 'normal' operation.
1226     if (t instanceof NotServingRegionException) {
1227       LOG.debug("NotServingRegionException; " +  t.getMessage());
1228       return t;
1229     }
1230     if (msg == null) {
1231       LOG.error("", RemoteExceptionHandler.checkThrowable(t));
1232     } else {
1233       LOG.error(msg, RemoteExceptionHandler.checkThrowable(t));
1234     }
1235     if (!checkOOME(t)) {
1236       checkFileSystem();
1237     }
1238     return t;
1239   }
1240 
1241   /*
1242    * @param t
1243    *
1244    * @return Make <code>t</code> an IOE if it isn't already.
1245    */
1246   private IOException convertThrowableToIOE(final Throwable t) {
1247     return convertThrowableToIOE(t, null);
1248   }
1249 
1250   /*
1251    * @param t
1252    *
1253    * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
1254    *
1255    * @return Make <code>t</code> an IOE if it isn't already.
1256    */
1257   private IOException convertThrowableToIOE(final Throwable t, final String msg) {
1258     return (t instanceof IOException ? (IOException) t : msg == null
1259         || msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
1260   }
1261 
1262   /*
1263    * Check if an OOME and, if so, abort immediately to avoid creating more objects.
1264    *
1265    * @param e
1266    *
1267    * @return True if we OOME'd and are aborting.
1268    */
1269   public boolean checkOOME(final Throwable e) {
1270     boolean stop = false;
1271     try {
1272       if (e instanceof OutOfMemoryError
1273           || (e.getCause() != null && e.getCause() instanceof OutOfMemoryError)
1274           || (e.getMessage() != null && e.getMessage().contains(
1275               "java.lang.OutOfMemoryError"))) {
1276         stop = true;
1277         LOG.fatal(
1278           "Run out of memory; HRegionServer will abort itself immediately", e);
1279       }
1280     } finally {
1281       if (stop) {
1282         Runtime.getRuntime().halt(1);
1283       }
1284     }
1285     return stop;
1286   }
1287 
1288   /**
1289    * Checks to see if the file system is still accessible. If not, sets
1290    * abortRequested and stopRequested
1291    *
1292    * @return false if file system is not available
1293    */
1294   public boolean checkFileSystem() {
1295     if (this.fsOk && this.fs != null) {
1296       try {
1297         FSUtils.checkFileSystemAvailable(this.fs);
1298       } catch (IOException e) {
1299         abort("File System not available", e);
1300         this.fsOk = false;
1301       }
1302     }
1303     return this.fsOk;
1304   }
1305 
1306   /*
1307    * Inner class that runs on a long period checking if regions need compaction.
1308    */
1309   private static class CompactionChecker extends Chore {
1310     private final HRegionServer instance;
1311     private final int majorCompactPriority;
1312     private final static int DEFAULT_PRIORITY = Integer.MAX_VALUE;
1313 
1314     CompactionChecker(final HRegionServer h, final int sleepTime,
1315         final Stoppable stopper) {
1316       super("CompactionChecker", sleepTime, h);
1317       this.instance = h;
1318       LOG.info("Runs every " + StringUtils.formatTime(sleepTime));
1319 
1320       /* MajorCompactPriority is configurable.
1321        * If not set, the compaction will use default priority.
1322        */
1323       this.majorCompactPriority = this.instance.conf.
1324         getInt("hbase.regionserver.compactionChecker.majorCompactPriority",
1325         DEFAULT_PRIORITY);
1326     }
1327 
1328     @Override
1329     protected void chore() {
1330       for (HRegion r : this.instance.onlineRegions.values()) {
1331         if (r == null)
1332           continue;
1333         for (Store s : r.getStores().values()) {
1334           try {
1335             if (s.needsCompaction()) {
1336               // Queue a compaction. Will recognize if major is needed.
1337               this.instance.compactSplitThread.requestCompaction(r, s, getName()
1338                   + " requests compaction", null);
1339             } else if (s.isMajorCompaction()) {
1340               if (majorCompactPriority == DEFAULT_PRIORITY
1341                   || majorCompactPriority > r.getCompactPriority()) {
1342                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1343                     + " requests major compaction; use default priority", null);
1344               } else {
1345                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1346                     + " requests major compaction; use configured priority",
1347                   this.majorCompactPriority, null);
1348               }
1349             }
1350           } catch (IOException e) {
1351             LOG.warn("Failed major compaction check on " + r, e);
1352           }
1353         }
1354       }
1355     }
1356   }
1357 
1358   class PeriodicMemstoreFlusher extends Chore {
1359     final HRegionServer server;
1360     final static int RANGE_OF_DELAY = 20000; //millisec
1361     final static int MIN_DELAY_TIME = 3000; //millisec
1362     public PeriodicMemstoreFlusher(int cacheFlushInterval, final HRegionServer server) {
1363       super(server.getServerName() + "-MemstoreFlusherChore", cacheFlushInterval, server);
1364       this.server = server;
1365     }
1366 
1367     @Override
1368     protected void chore() {
1369       for (HRegion r : this.server.onlineRegions.values()) {
1370         if (r == null)
1371           continue;
1372         if (r.shouldFlush()) {
1373           FlushRequester requester = server.getFlushRequester();
1374           if (requester != null) {
1375             long randomDelay = rand.nextInt(RANGE_OF_DELAY) + MIN_DELAY_TIME;
1376             LOG.info(getName() + " requesting flush for region " + r.getRegionNameAsString() + 
1377                 " after a delay of " + randomDelay);
1378             //Throttle the flushes by putting a delay. If we don't throttle, and there
1379             //is a balanced write-load on the regions in a table, we might end up 
1380             //overwhelming the filesystem with too many flushes at once.
1381             requester.requestDelayedFlush(r, randomDelay);
1382           }
1383         }
1384       }
1385     }
1386   }
1387 
1388   /**
1389    * Report the status of the server. A server is online once all the startup is
1390    * completed (setting up filesystem, starting service threads, etc.). This
1391    * method is designed mostly to be useful in tests.
1392    *
1393    * @return true if online, false if not.
1394    */
1395   public boolean isOnline() {
1396     return isOnline;
1397   }
1398 
1399   /**
1400    * Setup WAL log and replication if enabled.
1401    * Replication setup is done in here because it wants to be hooked up to WAL.
1402    * @return A WAL instance.
1403    * @throws IOException
1404    */
1405   private HLog setupWALAndReplication() throws IOException {
1406     final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1407     Path logdir = new Path(rootDir,
1408       HLog.getHLogDirectoryName(this.serverNameFromMasterPOV.toString()));
1409     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1410     if (this.fs.exists(logdir)) {
1411       throw new RegionServerRunningException("Region server has already " +
1412         "created directory at " + this.serverNameFromMasterPOV.toString());
1413     }
1414 
1415     // Instantiate replication manager if replication enabled.  Pass it the
1416     // log directories.
1417     createNewReplicationInstance(conf, this, this.fs, logdir, oldLogDir);
1418     return instantiateHLog(logdir, oldLogDir);
1419   }
1420 
1421   // The method is synchronized to guarantee atomic update to hlogForMeta - 
1422   // It is possible that multiple calls could be made to this method almost 
1423   // at the same time, one for _ROOT_ and another for .META. (if they happen
1424   // to be assigned to the same RS). Also, we want to use the same log for both
1425   private synchronized HLog getMetaWAL() throws IOException {
1426     if (this.hlogForMeta == null) {
1427       final String logName
1428       = HLog.getHLogDirectoryName(this.serverNameFromMasterPOV.toString());
1429 
1430       Path logdir = new Path(rootDir, logName);
1431       final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1432       if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1433       this.hlogForMeta = new HLog(this.fs.getBackingFs(), logdir, oldLogDir, this.conf,
1434           getMetaWALActionListeners(), false, this.serverNameFromMasterPOV.toString(), true);
1435     }
1436     return this.hlogForMeta;
1437   }
1438 
1439   /**
1440    * Called by {@link #setupWALAndReplication()} creating WAL instance.
1441    * @param logdir
1442    * @param oldLogDir
1443    * @return WAL instance.
1444    * @throws IOException
1445    */
1446   protected HLog instantiateHLog(Path logdir, Path oldLogDir) throws IOException {
1447     return new HLog(this.fs.getBackingFs(), logdir, oldLogDir, this.conf,
1448       getWALActionListeners(), this.serverNameFromMasterPOV.toString());
1449   }
1450 
1451   /**
1452    * Called by {@link #instantiateHLog(Path, Path)} setting up WAL instance.
1453    * Add any {@link WALActionsListener}s you want inserted before WAL startup.
1454    * @return List of WALActionsListener that will be passed in to
1455    * {@link HLog} on construction.
1456    */
1457   protected List<WALActionsListener> getWALActionListeners() {
1458     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1459     // Log roller.
1460     this.hlogRoller = new LogRoller(this, this);
1461     listeners.add(this.hlogRoller);
1462     if (this.replicationSourceHandler != null &&
1463         this.replicationSourceHandler.getWALActionsListener() != null) {
1464       // Replication handler is an implementation of WALActionsListener.
1465       listeners.add(this.replicationSourceHandler.getWALActionsListener());
1466     }
1467     return listeners;
1468   }
1469 
1470   protected List<WALActionsListener> getMetaWALActionListeners() {
1471     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1472     // Using a tmp log roller to ensure metaLogRoller is alive once it is not
1473     // null (addendum patch on HBASE-7213)
1474     MetaLogRoller tmpLogRoller = new MetaLogRoller(this, this);
1475     String n = Thread.currentThread().getName();
1476     Threads.setDaemonThreadRunning(tmpLogRoller.getThread(),
1477         n + "MetaLogRoller", uncaughtExceptionHandler);
1478     this.metaHLogRoller = tmpLogRoller;
1479     tmpLogRoller = null;
1480     listeners.add(this.metaHLogRoller);
1481     return listeners;
1482   }
1483 
1484   protected LogRoller getLogRoller() {
1485     return hlogRoller;
1486   }
1487 
1488   /*
1489    * @param interval Interval since last time metrics were called.
1490    */
1491   protected void doMetrics() {
1492     try {
1493       metrics();
1494     } catch (Throwable e) {
1495       LOG.warn("Failed metrics", e);
1496     }
1497   }
1498 
1499   protected void metrics() {
1500     this.metrics.regions.set(this.onlineRegions.size());
1501     this.metrics.incrementRequests(this.requestCount.get());
1502     this.metrics.requests.intervalHeartBeat();
1503     // Is this too expensive every three seconds getting a lock on onlineRegions
1504     // and then per store carried? Can I make metrics be sloppier and avoid
1505     // the synchronizations?
1506     int stores = 0;
1507     int storefiles = 0;
1508     long memstoreSize = 0;
1509     int readRequestsCount = 0;
1510     int writeRequestsCount = 0;
1511     long storefileIndexSize = 0;
1512     HDFSBlocksDistribution hdfsBlocksDistribution =
1513       new HDFSBlocksDistribution();
1514     long totalStaticIndexSize = 0;
1515     long totalStaticBloomSize = 0;
1516     long numPutsWithoutWAL = 0;
1517     long dataInMemoryWithoutWAL = 0;
1518     long updatesBlockedMs = 0;
1519 
1520     // Note that this is a map of Doubles instead of Longs. This is because we
1521     // do effective integer division, which would perhaps truncate more than it
1522     // should because we do it only on one part of our sum at a time. Rather
1523     // than dividing at the end, where it is difficult to know the proper
1524     // factor, everything is exact then truncated.
1525     final Map<String, MutableDouble> tempVals =
1526         new HashMap<String, MutableDouble>();
1527 
1528     for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
1529       HRegion r = e.getValue();
1530       memstoreSize += r.memstoreSize.get();
1531       numPutsWithoutWAL += r.numPutsWithoutWAL.get();
1532       dataInMemoryWithoutWAL += r.dataInMemoryWithoutWAL.get();
1533       readRequestsCount += r.readRequestsCount.get();
1534       writeRequestsCount += r.writeRequestsCount.get();
1535       updatesBlockedMs += r.updatesBlockedMs.get();
1536       synchronized (r.stores) {
1537         stores += r.stores.size();
1538         for (Map.Entry<byte[], Store> ee : r.stores.entrySet()) {
1539             final Store store = ee.getValue();
1540             final SchemaMetrics schemaMetrics = store.getSchemaMetrics();
1541 
1542             {
1543               long tmpStorefiles = store.getStorefilesCount();
1544               schemaMetrics.accumulateStoreMetric(tempVals,
1545                   StoreMetricType.STORE_FILE_COUNT, tmpStorefiles);
1546               storefiles += tmpStorefiles;
1547             }
1548 
1549 
1550             {
1551               long tmpStorefileIndexSize = store.getStorefilesIndexSize();
1552               schemaMetrics.accumulateStoreMetric(tempVals,
1553                   StoreMetricType.STORE_FILE_INDEX_SIZE,
1554                   (long) (tmpStorefileIndexSize / (1024.0 * 1024)));
1555               storefileIndexSize += tmpStorefileIndexSize;
1556             }
1557 
1558             {
1559               long tmpStorefilesSize = store.getStorefilesSize();
1560               schemaMetrics.accumulateStoreMetric(tempVals,
1561                   StoreMetricType.STORE_FILE_SIZE_MB,
1562                   (long) (tmpStorefilesSize / (1024.0 * 1024)));
1563             }
1564 
1565             {
1566               long tmpStaticBloomSize = store.getTotalStaticBloomSize();
1567               schemaMetrics.accumulateStoreMetric(tempVals,
1568                   StoreMetricType.STATIC_BLOOM_SIZE_KB,
1569                   (long) (tmpStaticBloomSize / 1024.0));
1570               totalStaticBloomSize += tmpStaticBloomSize;
1571             }
1572 
1573             {
1574               long tmpStaticIndexSize = store.getTotalStaticIndexSize();
1575               schemaMetrics.accumulateStoreMetric(tempVals,
1576                   StoreMetricType.STATIC_INDEX_SIZE_KB,
1577                   (long) (tmpStaticIndexSize / 1024.0));
1578               totalStaticIndexSize += tmpStaticIndexSize;
1579             }
1580 
1581             schemaMetrics.accumulateStoreMetric(tempVals,
1582                 StoreMetricType.MEMSTORE_SIZE_MB,
1583                 (long) (store.getMemStoreSize() / (1024.0 * 1024)));
1584         }
1585       }
1586 
1587       hdfsBlocksDistribution.add(r.getHDFSBlocksDistribution());
1588     }
1589 
1590     for (Entry<String, MutableDouble> e : tempVals.entrySet()) {
1591       RegionMetricsStorage.setNumericMetric(e.getKey(), e.getValue().longValue());
1592     }
1593 
1594     this.metrics.stores.set(stores);
1595     this.metrics.storefiles.set(storefiles);
1596     this.metrics.hlogFileCount.set(this.hlog.getNumLogFiles());
1597     this.metrics.hlogFileSizeMB.set(this.hlog.getNumLogFileSize() /(1024 * 1024));
1598     this.metrics.memstoreSizeMB.set((int) (memstoreSize / (1024 * 1024)));
1599     this.metrics.mbInMemoryWithoutWAL.set((int) (dataInMemoryWithoutWAL / (1024 * 1024)));
1600     this.metrics.numPutsWithoutWAL.set(numPutsWithoutWAL);
1601     this.metrics.storefileIndexSizeMB.set(
1602         (int) (storefileIndexSize / (1024 * 1024)));
1603     this.metrics.rootIndexSizeKB.set(
1604         (int) (storefileIndexSize / 1024));
1605     this.metrics.totalStaticIndexSizeKB.set(
1606         (int) (totalStaticIndexSize / 1024));
1607     this.metrics.totalStaticBloomSizeKB.set(
1608         (int) (totalStaticBloomSize / 1024));
1609     this.metrics.readRequestsCount.set(readRequestsCount);
1610     this.metrics.writeRequestsCount.set(writeRequestsCount);
1611     this.metrics.compactionQueueSize.set(compactSplitThread
1612         .getCompactionQueueSize());
1613     this.metrics.flushQueueSize.set(cacheFlusher
1614         .getFlushQueueSize());
1615     this.metrics.updatesBlockedSeconds.set(updatesBlockedMs/1000);
1616     final long updatesBlockedMsHigherWater = cacheFlusher.getUpdatesBlockedMsHighWater().get();
1617     this.metrics.updatesBlockedSecondsHighWater.set(updatesBlockedMsHigherWater/1000);
1618 
1619     BlockCache blockCache = cacheConfig.getBlockCache();
1620     if (blockCache != null) {
1621       this.metrics.blockCacheCount.set(blockCache.size());
1622       this.metrics.blockCacheFree.set(blockCache.getFreeSize());
1623       this.metrics.blockCacheSize.set(blockCache.getCurrentSize());
1624       CacheStats cacheStats = blockCache.getStats();
1625       this.metrics.blockCacheHitCount.set(cacheStats.getHitCount());
1626       this.metrics.blockCacheMissCount.set(cacheStats.getMissCount());
1627       this.metrics.blockCacheEvictedCount.set(blockCache.getEvictedCount());
1628       double ratio = blockCache.getStats().getHitRatio();
1629       int percent = (int) (ratio * 100);
1630       this.metrics.blockCacheHitRatio.set(percent);
1631       ratio = blockCache.getStats().getHitCachingRatio();
1632       percent = (int) (ratio * 100);
1633       this.metrics.blockCacheHitCachingRatio.set(percent);
1634       // past N period block cache hit / hit caching ratios
1635       cacheStats.rollMetricsPeriod();
1636       ratio = cacheStats.getHitRatioPastNPeriods();
1637       percent = (int) (ratio * 100);
1638       this.metrics.blockCacheHitRatioPastNPeriods.set(percent);
1639       ratio = cacheStats.getHitCachingRatioPastNPeriods();
1640       percent = (int) (ratio * 100);
1641       this.metrics.blockCacheHitCachingRatioPastNPeriods.set(percent);
1642     }
1643     float localityIndex = hdfsBlocksDistribution.getBlockLocalityIndex(
1644       getServerName().getHostname());
1645     int percent = (int) (localityIndex * 100);
1646     this.metrics.hdfsBlocksLocalityIndex.set(percent);
1647 
1648   }
1649 
1650   /**
1651    * @return Region server metrics instance.
1652    */
1653   public RegionServerMetrics getMetrics() {
1654     return this.metrics;
1655   }
1656 
1657   /**
1658    * @return Master address tracker instance.
1659    */
1660   public MasterAddressTracker getMasterAddressManager() {
1661     return this.masterAddressManager;
1662   }
1663 
1664   /*
1665    * Start maintanence Threads, Server, Worker and lease checker threads.
1666    * Install an UncaughtExceptionHandler that calls abort of RegionServer if we
1667    * get an unhandled exception. We cannot set the handler on all threads.
1668    * Server's internal Listener thread is off limits. For Server, if an OOME, it
1669    * waits a while then retries. Meantime, a flush or a compaction that tries to
1670    * run should trigger same critical condition and the shutdown will run. On
1671    * its way out, this server will shut down Server. Leases are sort of
1672    * inbetween. It has an internal thread that while it inherits from Chore, it
1673    * keeps its own internal stop mechanism so needs to be stopped by this
1674    * hosting server. Worker logs the exception and exits.
1675    */
1676   private void startServiceThreads() throws IOException {
1677     String n = Thread.currentThread().getName();
1678     // Start executor services
1679     this.service = new ExecutorService(getServerName().toString());
1680     this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
1681       conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1682     this.service.startExecutorService(ExecutorType.RS_OPEN_ROOT,
1683       conf.getInt("hbase.regionserver.executor.openroot.threads", 1));
1684     this.service.startExecutorService(ExecutorType.RS_OPEN_META,
1685       conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1686     this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1687       conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1688     this.service.startExecutorService(ExecutorType.RS_CLOSE_ROOT,
1689       conf.getInt("hbase.regionserver.executor.closeroot.threads", 1));
1690     this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
1691       conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1692 
1693     Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), n + ".logRoller",
1694         uncaughtExceptionHandler);
1695     Threads.setDaemonThreadRunning(this.cacheFlusher.getThread(), n + ".cacheFlusher",
1696         uncaughtExceptionHandler);
1697     Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), n +
1698       ".compactionChecker", uncaughtExceptionHandler);
1699     Threads.setDaemonThreadRunning(this.periodicFlusher.getThread(), n +
1700         ".periodicFlusher", uncaughtExceptionHandler);
1701     if (this.healthCheckChore != null) {
1702       Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker",
1703           uncaughtExceptionHandler);
1704     }
1705 
1706     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
1707     // an unhandled exception, it will just exit.
1708     this.leases.setName(n + ".leaseChecker");
1709     this.leases.start();
1710 
1711     // Put up the webui.  Webui may come up on port other than configured if
1712     // that port is occupied. Adjust serverInfo if this is the case.
1713     this.webuiport = putUpWebUI();
1714 
1715     if (this.replicationSourceHandler == this.replicationSinkHandler &&
1716         this.replicationSourceHandler != null) {
1717       this.replicationSourceHandler.startReplicationService();
1718     } else {
1719       if (this.replicationSourceHandler != null) {
1720         this.replicationSourceHandler.startReplicationService();
1721       }
1722       if (this.replicationSinkHandler != null) {
1723         this.replicationSinkHandler.startReplicationService();
1724       }
1725     }
1726 
1727     // Start Server.  This service is like leases in that it internally runs
1728     // a thread.
1729     this.rpcServer.start();
1730 
1731     // Create the log splitting worker and start it
1732     this.splitLogWorker = new SplitLogWorker(this.zooKeeper,
1733         this.getConfiguration(), this.getServerName().toString());
1734     splitLogWorker.start();
1735     
1736   }
1737 
1738   /**
1739    * Puts up the webui.
1740    * @return Returns final port -- maybe different from what we started with.
1741    * @throws IOException
1742    */
1743   private int putUpWebUI() throws IOException {
1744     int port = this.conf.getInt("hbase.regionserver.info.port", 60030);
1745     // -1 is for disabling info server
1746     if (port < 0) return port;
1747     String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
1748     // check if auto port bind enabled
1749     boolean auto = this.conf.getBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO,
1750         false);
1751     while (true) {
1752       try {
1753         this.infoServer = new InfoServer("regionserver", addr, port, false, this.conf);
1754         this.infoServer.addServlet("status", "/rs-status", RSStatusServlet.class);
1755         this.infoServer.addServlet("dump", "/dump", RSDumpServlet.class);
1756         this.infoServer.setAttribute(REGIONSERVER, this);
1757         this.infoServer.setAttribute(REGIONSERVER_CONF, conf);
1758         this.infoServer.start();
1759         break;
1760       } catch (BindException e) {
1761         if (!auto) {
1762           // auto bind disabled throw BindException
1763           throw e;
1764         }
1765         // auto bind enabled, try to use another port
1766         LOG.info("Failed binding http info server to port: " + port);
1767         port++;
1768       }
1769     }
1770     return port;
1771   }
1772 
1773   /*
1774    * Verify that server is healthy
1775    */
1776   private boolean isHealthy() {
1777     if (!fsOk) {
1778       // File system problem
1779       return false;
1780     }
1781     // Verify that all threads are alive
1782     if (!(leases.isAlive()
1783         && cacheFlusher.isAlive() && hlogRoller.isAlive()
1784         && this.compactionChecker.isAlive())
1785         && this.periodicFlusher.isAlive()) {
1786       stop("One or more threads are no longer alive -- stop");
1787       return false;
1788     }
1789     if (metaHLogRoller != null && !metaHLogRoller.isAlive()) {
1790       stop("Meta HLog roller thread is no longer alive -- stop");
1791       return false;
1792     }
1793     return true;
1794   }
1795 
1796   public HLog getWAL() {
1797     try {
1798       return getWAL(null);
1799     } catch (IOException e) {
1800       LOG.warn("getWAL threw exception " + e);
1801       return null; 
1802     }
1803   }
1804 
1805   @Override
1806   public HLog getWAL(HRegionInfo regionInfo) throws IOException {
1807     //TODO: at some point this should delegate to the HLogFactory
1808     //currently, we don't care about the region as much as we care about the 
1809     //table.. (hence checking the tablename below)
1810     //_ROOT_ and .META. regions have separate WAL. 
1811     if (this.separateHLogForMeta && 
1812         regionInfo != null && 
1813         regionInfo.isMetaTable()) {
1814       return getMetaWAL();
1815     }
1816     return this.hlog;
1817   }
1818 
1819   @Override
1820   public CatalogTracker getCatalogTracker() {
1821     return this.catalogTracker;
1822   }
1823 
1824   @Override
1825   public void stop(final String msg) {
1826     try {
1827       if (this.rsHost != null) {
1828         this.rsHost.preStop(msg);
1829       }
1830       this.stopped = true;
1831       LOG.info("STOPPED: " + msg);
1832       // Wakes run() if it is sleeping
1833       sleeper.skipSleepCycle();
1834     } catch (IOException exp) {
1835       LOG.warn("The region server did not stop", exp);
1836     }
1837   }
1838 
1839   public void waitForServerOnline(){
1840     while (!isOnline() && !isStopped()){
1841        sleeper.sleep();
1842     }
1843   }
1844 
1845   @Override
1846   public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct,
1847       final boolean daughter)
1848   throws KeeperException, IOException {
1849     checkOpen();
1850     LOG.info("Post open deploy tasks for region=" + r.getRegionNameAsString() +
1851       ", daughter=" + daughter);
1852     // Do checks to see if we need to compact (references or too many files)
1853     for (Store s : r.getStores().values()) {
1854       if (s.hasReferences() || s.needsCompaction()) {
1855         getCompactionRequester().requestCompaction(r, s, "Opening Region", null);
1856       }
1857     }
1858     // Update ZK, ROOT or META
1859     if (r.getRegionInfo().isRootRegion()) {
1860       RootLocationEditor.setRootLocation(getZooKeeper(),
1861        this.serverNameFromMasterPOV);
1862     } else if (r.getRegionInfo().isMetaRegion()) {
1863       MetaEditor.updateMetaLocation(ct, r.getRegionInfo(),
1864         this.serverNameFromMasterPOV);
1865     } else {
1866       if (daughter) {
1867         // If daughter of a split, update whole row, not just location.
1868         MetaEditor.addDaughter(ct, r.getRegionInfo(),
1869           this.serverNameFromMasterPOV);
1870       } else {
1871         MetaEditor.updateRegionLocation(ct, r.getRegionInfo(),
1872           this.serverNameFromMasterPOV);
1873       }
1874     }
1875     LOG.info("Done with post open deploy task for region=" +
1876       r.getRegionNameAsString() + ", daughter=" + daughter);
1877 
1878   }
1879 
1880   /**
1881    * Return a reference to the metrics instance used for counting RPC calls.
1882    * @return Metrics instance.
1883    */
1884   public HBaseRpcMetrics getRpcMetrics() {
1885     return rpcServer.getRpcMetrics();
1886   }
1887 
1888   @Override
1889   public RpcServer getRpcServer() {
1890     return rpcServer;
1891   }
1892 
1893   /**
1894    * Cause the server to exit without closing the regions it is serving, the log
1895    * it is using and without notifying the master. Used unit testing and on
1896    * catastrophic events such as HDFS is yanked out from under hbase or we OOME.
1897    *
1898    * @param reason
1899    *          the reason we are aborting
1900    * @param cause
1901    *          the exception that caused the abort, or null
1902    */
1903   public void abort(String reason, Throwable cause) {
1904     String msg = "ABORTING region server " + this + ": " + reason;
1905     if (cause != null) {
1906       LOG.fatal(msg, cause);
1907     } else {
1908       LOG.fatal(msg);
1909     }
1910     this.abortRequested = true;
1911     this.reservedSpace.clear();
1912     // HBASE-4014: show list of coprocessors that were loaded to help debug
1913     // regionserver crashes.Note that we're implicitly using
1914     // java.util.HashSet's toString() method to print the coprocessor names.
1915     LOG.fatal("RegionServer abort: loaded coprocessors are: " +
1916         CoprocessorHost.getLoadedCoprocessors());
1917     if (this.metrics != null) {
1918       LOG.info("Dump of metrics: " + this.metrics);
1919     }
1920     // Do our best to report our abort to the master, but this may not work
1921     try {
1922       if (cause != null) {
1923         msg += "\nCause:\n" + StringUtils.stringifyException(cause);
1924       }
1925       if (hbaseMaster != null) {
1926         hbaseMaster.reportRSFatalError(
1927             this.serverNameFromMasterPOV.getVersionedBytes(), msg);
1928       }
1929     } catch (Throwable t) {
1930       LOG.warn("Unable to report fatal error to master", t);
1931     }
1932     stop(reason);
1933   }
1934 
1935   /**
1936    * @see HRegionServer#abort(String, Throwable)
1937    */
1938   public void abort(String reason) {
1939     abort(reason, null);
1940   }
1941 
1942   public boolean isAborted() {
1943     return this.abortRequested;
1944   }
1945 
1946   /*
1947    * Simulate a kill -9 of this server. Exits w/o closing regions or cleaninup
1948    * logs but it does close socket in case want to bring up server on old
1949    * hostname+port immediately.
1950    */
1951   protected void kill() {
1952     this.killed = true;
1953     abort("Simulated kill");
1954   }
1955 
1956   /**
1957    * Wait on all threads to finish. Presumption is that all closes and stops
1958    * have already been called.
1959    */
1960   protected void join() {
1961     Threads.shutdown(this.compactionChecker.getThread());
1962     Threads.shutdown(this.periodicFlusher.getThread());
1963     Threads.shutdown(this.cacheFlusher.getThread());
1964     if (this.healthCheckChore != null) {
1965       Threads.shutdown(this.healthCheckChore.getThread());
1966     }
1967     if (this.hlogRoller != null) {
1968       Threads.shutdown(this.hlogRoller.getThread());
1969     }
1970     if (this.metaHLogRoller != null) {
1971       Threads.shutdown(this.metaHLogRoller.getThread());
1972     }
1973     if (this.compactSplitThread != null) {
1974       this.compactSplitThread.join();
1975     }
1976     if (this.service != null) this.service.shutdown();
1977     if (this.replicationSourceHandler != null &&
1978         this.replicationSourceHandler == this.replicationSinkHandler) {
1979       this.replicationSourceHandler.stopReplicationService();
1980     } else {
1981       if (this.replicationSourceHandler != null) {
1982         this.replicationSourceHandler.stopReplicationService();
1983       }
1984       if (this.replicationSinkHandler != null) {
1985         this.replicationSinkHandler.stopReplicationService();
1986       }
1987     }
1988   }
1989 
1990   /**
1991    * @return Return the object that implements the replication
1992    * source service.
1993    */
1994   ReplicationSourceService getReplicationSourceService() {
1995     return replicationSourceHandler;
1996   }
1997 
1998   /**
1999    * @return Return the object that implements the replication
2000    * sink service.
2001    */
2002   ReplicationSinkService getReplicationSinkService() {
2003     return replicationSinkHandler;
2004   }
2005 
2006   /**
2007    * Get the current master from ZooKeeper and open the RPC connection to it.
2008    *
2009    * Method will block until a master is available. You can break from this
2010    * block by requesting the server stop.
2011    *
2012    * @return master + port, or null if server has been stopped
2013    */
2014   private ServerName getMaster() {
2015     ServerName masterServerName = null;
2016     long previousLogTime = 0;
2017     HMasterRegionInterface master = null;
2018     InetSocketAddress masterIsa = null;
2019     while (keepLooping() && master == null) {
2020       masterServerName = this.masterAddressManager.getMasterAddress();
2021       if (masterServerName == null) {
2022         if (!keepLooping()) {
2023           // give up with no connection.
2024           LOG.debug("No master found and cluster is stopped; bailing out");
2025           return null;
2026         }
2027         LOG.debug("No master found; retry");
2028         previousLogTime = System.currentTimeMillis();
2029 
2030         sleeper.sleep();
2031         continue;
2032       }
2033 
2034       masterIsa =
2035         new InetSocketAddress(masterServerName.getHostname(), masterServerName.getPort());
2036 
2037       LOG.info("Attempting connect to Master server at " + masterServerName);
2038       try {
2039         // Do initial RPC setup. The final argument indicates that the RPC
2040         // should retry indefinitely.
2041         master = HBaseRPC.waitForProxy(this.rpcEngine,
2042             HMasterRegionInterface.class, HMasterRegionInterface.VERSION,
2043             masterIsa, this.conf, -1,
2044             this.rpcTimeout, this.rpcTimeout);
2045       } catch (IOException e) {
2046         e = e instanceof RemoteException ?
2047             ((RemoteException)e).unwrapRemoteException() : e;
2048         if (e instanceof ServerNotRunningYetException) {
2049           if (System.currentTimeMillis() > (previousLogTime+1000)){
2050             LOG.info("Master isn't available yet, retrying");
2051             previousLogTime = System.currentTimeMillis();
2052           }
2053         } else {
2054           if (System.currentTimeMillis() > (previousLogTime + 1000)) {
2055             LOG.warn("Unable to connect to master. Retrying. Error was:", e);
2056             previousLogTime = System.currentTimeMillis();
2057           }
2058         }
2059         try {
2060           Thread.sleep(200);
2061         } catch (InterruptedException ignored) {
2062         }
2063       }
2064     }
2065     LOG.info("Connected to master at " + masterIsa);
2066     this.hbaseMaster = master;
2067     return masterServerName;
2068   }
2069 
2070   /**
2071    * @return True if we should break loop because cluster is going down or
2072    * this server has been stopped or hdfs has gone bad.
2073    */
2074   private boolean keepLooping() {
2075     return !this.stopped && isClusterUp();
2076   }
2077 
2078   /*
2079    * Let the master know we're here Run initialization using parameters passed
2080    * us by the master.
2081    * @return A Map of key/value configurations we got from the Master else
2082    * null if we failed to register.
2083    * @throws IOException
2084    */
2085   private MapWritable reportForDuty() throws IOException {
2086     MapWritable result = null;
2087     ServerName masterServerName = getMaster();
2088     if (masterServerName == null) return result;
2089     try {
2090       this.requestCount.set(0);
2091       LOG.info("Telling master at " + masterServerName + " that we are up " +
2092         "with port=" + this.isa.getPort() + ", startcode=" + this.startcode);
2093       long now = EnvironmentEdgeManager.currentTimeMillis();
2094       int port = this.isa.getPort();
2095       result = this.hbaseMaster.regionServerStartup(port, this.startcode, now);
2096     } catch (RemoteException e) {
2097       IOException ioe = e.unwrapRemoteException();
2098       if (ioe instanceof ClockOutOfSyncException) {
2099         LOG.fatal("Master rejected startup because clock is out of sync", ioe);
2100         // Re-throw IOE will cause RS to abort
2101         throw ioe;
2102       } else {
2103         LOG.warn("remote error telling master we are up", e);
2104       }
2105     } catch (IOException e) {
2106       LOG.warn("error telling master we are up", e);
2107     }
2108     return result;
2109   }
2110 
2111   /**
2112    * Closes all regions.  Called on our way out.
2113    * Assumes that its not possible for new regions to be added to onlineRegions
2114    * while this method runs.
2115    */
2116   protected void closeAllRegions(final boolean abort) {
2117     closeUserRegions(abort);
2118     closeMetaTableRegions(abort);
2119   }
2120 
2121   /**
2122    * Close root and meta regions if we carry them
2123    * @param abort Whether we're running an abort.
2124    */
2125   void closeMetaTableRegions(final boolean abort) {
2126     HRegion meta = null;
2127     HRegion root = null;
2128     this.lock.writeLock().lock();
2129     try {
2130       for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
2131         HRegionInfo hri = e.getValue().getRegionInfo();
2132         if (hri.isRootRegion()) {
2133           root = e.getValue();
2134         } else if (hri.isMetaRegion()) {
2135           meta = e.getValue();
2136         }
2137         if (meta != null && root != null) break;
2138       }
2139     } finally {
2140       this.lock.writeLock().unlock();
2141     }
2142     if (meta != null) closeRegion(meta.getRegionInfo(), abort, false);
2143     if (root != null) closeRegion(root.getRegionInfo(), abort, false);
2144   }
2145 
2146   /**
2147    * Schedule closes on all user regions.
2148    * Should be safe calling multiple times because it wont' close regions
2149    * that are already closed or that are closing.
2150    * @param abort Whether we're running an abort.
2151    */
2152   void closeUserRegions(final boolean abort) {
2153     this.lock.writeLock().lock();
2154     try {
2155       for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
2156         HRegion r = e.getValue();
2157         if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
2158           // Don't update zk with this close transition; pass false.
2159           closeRegion(r.getRegionInfo(), abort, false);
2160         }
2161       }
2162     } finally {
2163       this.lock.writeLock().unlock();
2164     }
2165   }
2166 
2167   @Override
2168   @QosPriority(priority=HConstants.HIGH_QOS)
2169   public HRegionInfo getRegionInfo(final byte[] regionName)
2170   throws NotServingRegionException, IOException {
2171     checkOpen();
2172     requestCount.incrementAndGet();
2173     return getRegion(regionName).getRegionInfo();
2174   }
2175 
2176   public Result getClosestRowBefore(final byte[] regionName, final byte[] row,
2177       final byte[] family) throws IOException {
2178     checkOpen();
2179     requestCount.incrementAndGet();
2180     try {
2181       // locate the region we're operating on
2182       HRegion region = getRegion(regionName);
2183       // ask the region for all the data
2184 
2185       Result r = region.getClosestRowBefore(row, family);
2186       return r;
2187     } catch (Throwable t) {
2188       throw convertThrowableToIOE(cleanup(t));
2189     }
2190   }
2191 
2192   /** {@inheritDoc} */
2193   public Result get(byte[] regionName, Get get) throws IOException {
2194     checkOpen();
2195     requestCount.incrementAndGet();
2196     try {
2197       HRegion region = getRegion(regionName);
2198       return region.get(get, getLockFromId(get.getLockId()));
2199     } catch (Throwable t) {
2200       throw convertThrowableToIOE(cleanup(t));
2201     }
2202   }
2203 
2204   public boolean exists(byte[] regionName, Get get) throws IOException {
2205     checkOpen();
2206     requestCount.incrementAndGet();
2207     try {
2208       HRegion region = getRegion(regionName);
2209       Integer lock = getLockFromId(get.getLockId());
2210       if (region.getCoprocessorHost() != null) {
2211         Boolean result = region.getCoprocessorHost().preExists(get);
2212         if (result != null) {
2213           return result.booleanValue();
2214         }
2215       }
2216       Result r = region.get(get, lock);
2217       boolean result = r != null && !r.isEmpty();
2218       if (region.getCoprocessorHost() != null) {
2219         result = region.getCoprocessorHost().postExists(get, result);
2220       }
2221       return result;
2222     } catch (Throwable t) {
2223       throw convertThrowableToIOE(cleanup(t));
2224     }
2225   }
2226 
2227   public void put(final byte[] regionName, final Put put) throws IOException {
2228     if (put.getRow() == null) {
2229       throw new IllegalArgumentException("update has null row");
2230     }
2231 
2232     checkOpen();
2233     this.requestCount.incrementAndGet();
2234     HRegion region = getRegion(regionName);
2235     try {
2236       if (!region.getRegionInfo().isMetaTable()) {
2237         this.cacheFlusher.reclaimMemStoreMemory();
2238       }
2239       boolean writeToWAL = put.getWriteToWAL();
2240       region.put(put, getLockFromId(put.getLockId()), writeToWAL);
2241     } catch (Throwable t) {
2242       throw convertThrowableToIOE(cleanup(t));
2243     }
2244   }
2245 
2246   public int put(final byte[] regionName, final List<Put> puts)
2247       throws IOException {
2248     checkOpen();
2249     HRegion region = null;
2250     int i = 0;
2251 
2252     try {
2253       region = getRegion(regionName);
2254       if (!region.getRegionInfo().isMetaTable()) {
2255         this.cacheFlusher.reclaimMemStoreMemory();
2256       }
2257 
2258       @SuppressWarnings("unchecked")
2259       Pair<Mutation, Integer>[] putsWithLocks = new Pair[puts.size()];
2260 
2261       for (Put p : puts) {
2262         Integer lock = getLockFromId(p.getLockId());
2263         putsWithLocks[i++] = new Pair<Mutation, Integer>(p, lock);
2264       }
2265 
2266       this.requestCount.addAndGet(puts.size());
2267       OperationStatus codes[] = region.batchMutate(putsWithLocks);
2268       for (i = 0; i < codes.length; i++) {
2269         if (codes[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
2270           return i;
2271         }
2272       }
2273       return -1;
2274     } catch (Throwable t) {
2275       throw convertThrowableToIOE(cleanup(t));
2276     }
2277   }
2278 
2279   private boolean checkAndMutate(final byte[] regionName, final byte[] row,
2280       final byte[] family, final byte[] qualifier, final CompareOp compareOp,
2281       final WritableByteArrayComparable comparator, final Writable w,
2282       Integer lock) throws IOException {
2283     checkOpen();
2284     this.requestCount.incrementAndGet();
2285     HRegion region = getRegion(regionName);
2286     try {
2287       if (!region.getRegionInfo().isMetaTable()) {
2288         this.cacheFlusher.reclaimMemStoreMemory();
2289       }
2290       return region.checkAndMutate(row, family, qualifier, compareOp,
2291         comparator, w, lock, true);
2292     } catch (Throwable t) {
2293       throw convertThrowableToIOE(cleanup(t));
2294     }
2295   }
2296 
2297   /**
2298    *
2299    * @param regionName
2300    * @param row
2301    * @param family
2302    * @param qualifier
2303    * @param value
2304    *          the expected value
2305    * @param put
2306    * @throws IOException
2307    * @return true if the new put was execute, false otherwise
2308    */
2309   public boolean checkAndPut(final byte[] regionName, final byte[] row,
2310       final byte[] family, final byte[] qualifier, final byte[] value,
2311       final Put put) throws IOException {
2312     checkOpen();
2313     if (regionName == null) {
2314       throw new IOException("Invalid arguments to checkAndPut "
2315           + "regionName is null");
2316     }
2317     HRegion region = getRegion(regionName);
2318     Integer lock = getLockFromId(put.getLockId());
2319     WritableByteArrayComparable comparator = new BinaryComparator(value);
2320     if (region.getCoprocessorHost() != null) {
2321       Boolean result = region.getCoprocessorHost()
2322         .preCheckAndPut(row, family, qualifier, CompareOp.EQUAL, comparator,
2323           put);
2324       if (result != null) {
2325         return result.booleanValue();
2326       }
2327     }
2328     boolean result = checkAndMutate(regionName, row, family, qualifier,
2329         CompareOp.EQUAL, comparator, put,
2330       lock);
2331     if (region.getCoprocessorHost() != null) {
2332       result = region.getCoprocessorHost().postCheckAndPut(row, family,
2333         qualifier, CompareOp.EQUAL, comparator, put, result);
2334     }
2335     return result;
2336   }
2337 
2338   /**
2339    *
2340    * @param regionName
2341    * @param row
2342    * @param family
2343    * @param qualifier
2344    * @param compareOp
2345    * @param comparator
2346    * @param put
2347    * @throws IOException
2348    * @return true if the new put was execute, false otherwise
2349    */
2350   public boolean checkAndPut(final byte[] regionName, final byte[] row,
2351       final byte[] family, final byte[] qualifier, final CompareOp compareOp,
2352       final WritableByteArrayComparable comparator, final Put put)
2353        throws IOException {
2354     checkOpen();
2355     if (regionName == null) {
2356       throw new IOException("Invalid arguments to checkAndPut "
2357           + "regionName is null");
2358     }
2359     HRegion region = getRegion(regionName);
2360     Integer lock = getLockFromId(put.getLockId());
2361     if (region.getCoprocessorHost() != null) {
2362       Boolean result = region.getCoprocessorHost()
2363         .preCheckAndPut(row, family, qualifier, compareOp, comparator, put);
2364       if (result != null) {
2365         return result.booleanValue();
2366       }
2367     }
2368     boolean result = checkAndMutate(regionName, row, family, qualifier,
2369       compareOp, comparator, put, lock);
2370     if (region.getCoprocessorHost() != null) {
2371       result = region.getCoprocessorHost().postCheckAndPut(row, family,
2372         qualifier, compareOp, comparator, put, result);
2373     }
2374     return result;
2375   }
2376 
2377   /**
2378    *
2379    * @param regionName
2380    * @param row
2381    * @param family
2382    * @param qualifier
2383    * @param value
2384    *          the expected value
2385    * @param delete
2386    * @throws IOException
2387    * @return true if the new put was execute, false otherwise
2388    */
2389   public boolean checkAndDelete(final byte[] regionName, final byte[] row,
2390       final byte[] family, final byte[] qualifier, final byte[] value,
2391       final Delete delete) throws IOException {
2392     checkOpen();
2393 
2394     if (regionName == null) {
2395       throw new IOException("Invalid arguments to checkAndDelete "
2396           + "regionName is null");
2397     }
2398     HRegion region = getRegion(regionName);
2399     Integer lock = getLockFromId(delete.getLockId());
2400     WritableByteArrayComparable comparator = new BinaryComparator(value);
2401     if (region.getCoprocessorHost() != null) {
2402       Boolean result = region.getCoprocessorHost().preCheckAndDelete(row,
2403         family, qualifier, CompareOp.EQUAL, comparator, delete);
2404       if (result != null) {
2405         return result.booleanValue();
2406       }
2407     }
2408     boolean result = checkAndMutate(regionName, row, family, qualifier,
2409       CompareOp.EQUAL, comparator, delete, lock);
2410     if (region.getCoprocessorHost() != null) {
2411       result = region.getCoprocessorHost().postCheckAndDelete(row, family,
2412         qualifier, CompareOp.EQUAL, comparator, delete, result);
2413     }
2414     return result;
2415   }
2416 
2417   @Override
2418   public List<String> getStoreFileList(byte[] regionName, byte[] columnFamily)
2419     throws IllegalArgumentException {
2420     return getStoreFileList(regionName, new byte[][]{columnFamily});
2421   }
2422 
2423   @Override
2424   public List<String> getStoreFileList(byte[] regionName, byte[][] columnFamilies)
2425     throws IllegalArgumentException {
2426     HRegion region = getOnlineRegion(regionName);
2427     if (region == null) {
2428       throw new IllegalArgumentException("No region: " + new String(regionName)
2429           + " available");
2430     }
2431     return region.getStoreFileList(columnFamilies);
2432   }
2433 
2434   public List<String> getStoreFileList(byte[] regionName)
2435     throws IllegalArgumentException {
2436     HRegion region = getOnlineRegion(regionName);
2437     if (region == null) {
2438       throw new IllegalArgumentException("No region: " + new String(regionName)
2439           + " available");
2440     }
2441     Set<byte[]> columnFamilies = region.getStores().keySet();
2442     int nCF = columnFamilies.size();
2443     return region.getStoreFileList(columnFamilies.toArray(new byte[nCF][]));
2444   }
2445   
2446  /**
2447   * Flushes the given region
2448   */
2449   public void flushRegion(byte[] regionName)
2450     throws IllegalArgumentException, IOException {
2451     HRegion region = getOnlineRegion(regionName);
2452     if (region == null) {
2453       throw new IllegalArgumentException("No region : " + new String(regionName)
2454       + " available");
2455     }
2456     boolean needsCompaction = region.flushcache();
2457     if (needsCompaction) {
2458       this.compactSplitThread.requestCompaction(region, "Compaction through user triggered flush");
2459     }
2460   }
2461 
2462  /**
2463    * Flushes the given region if lastFlushTime < ifOlderThanTS
2464    */
2465    public void flushRegion(byte[] regionName, long ifOlderThanTS)
2466      throws IllegalArgumentException, IOException {
2467      HRegion region = getOnlineRegion(regionName);
2468      if (region == null) {
2469        throw new IllegalArgumentException("No region : " + new String(regionName)
2470        + " available");
2471      }
2472      if (region.getLastFlushTime() < ifOlderThanTS) {
2473       boolean needsCompaction = region.flushcache();
2474       if (needsCompaction) {
2475         this.compactSplitThread
2476             .requestCompaction(region, "Compaction through user triggered flush");
2477       }
2478     }
2479    }
2480 
2481   /**
2482    * Gets last flush time for the given region
2483    * @return the last flush time for a region
2484    */
2485   public long getLastFlushTime(byte[] regionName) {
2486     HRegion region = getOnlineRegion(regionName);
2487     if (region == null) {
2488       throw new IllegalArgumentException("No region : " + new String(regionName)
2489       + " available");
2490     }
2491     return region.getLastFlushTime();
2492   }
2493  
2494   /**
2495    *
2496    * @param regionName
2497    * @param row
2498    * @param family
2499    * @param qualifier
2500    * @param compareOp
2501    * @param comparator
2502    * @param delete
2503    * @throws IOException
2504    * @return true if the new put was execute, false otherwise
2505    */
2506   public boolean checkAndDelete(final byte[] regionName, final byte[] row,
2507       final byte[] family, final byte[] qualifier, final CompareOp compareOp,
2508       final WritableByteArrayComparable comparator, final Delete delete)
2509       throws IOException {
2510     checkOpen();
2511 
2512     if (regionName == null) {
2513       throw new IOException("Invalid arguments to checkAndDelete "
2514         + "regionName is null");
2515     }
2516     HRegion region = getRegion(regionName);
2517     Integer lock = getLockFromId(delete.getLockId());
2518     if (region.getCoprocessorHost() != null) {
2519       Boolean result = region.getCoprocessorHost().preCheckAndDelete(row,
2520         family, qualifier, compareOp, comparator, delete);
2521      if (result != null) {
2522        return result.booleanValue();
2523      }
2524     }
2525     boolean result = checkAndMutate(regionName, row, family, qualifier,
2526       compareOp, comparator, delete, lock);
2527    if (region.getCoprocessorHost() != null) {
2528      result = region.getCoprocessorHost().postCheckAndDelete(row, family,
2529        qualifier, compareOp, comparator, delete, result);
2530    }
2531    return result;
2532  }
2533 
2534   //
2535   // remote scanner interface
2536   //
2537   
2538   public long openScanner(byte[] regionName, Scan scan) throws IOException {
2539     RegionScanner s = internalOpenScanner(regionName, scan);
2540     long scannerId = addScanner(s);
2541     return scannerId;
2542   }
2543 
2544   private RegionScanner internalOpenScanner(byte[] regionName, Scan scan)
2545       throws IOException {
2546     checkOpen();
2547     NullPointerException npe = null;
2548     if (regionName == null) {
2549       npe = new NullPointerException("regionName is null");
2550     } else if (scan == null) {
2551       npe = new NullPointerException("scan is null");
2552     }
2553     if (npe != null) {
2554       throw new IOException("Invalid arguments to openScanner", npe);
2555     }
2556     requestCount.incrementAndGet();
2557     try {
2558       HRegion r = getRegion(regionName);
2559       r.checkRow(scan.getStartRow(), "Scan");
2560       scan.setLoadColumnFamiliesOnDemand(r.isLoadingCfsOnDemandDefault()
2561           || scan.doLoadColumnFamiliesOnDemand());
2562       r.prepareScanner(scan);
2563       RegionScanner s = null;
2564       if (r.getCoprocessorHost() != null) {
2565         s = r.getCoprocessorHost().preScannerOpen(scan);
2566       }
2567       if (s == null) {
2568         s = r.getScanner(scan);
2569       }
2570       if (r.getCoprocessorHost() != null) {
2571         RegionScanner savedScanner = r.getCoprocessorHost().postScannerOpen(
2572             scan, s);
2573         if (savedScanner == null) {
2574           LOG.warn("PostScannerOpen impl returning null. "
2575               + "Check the RegionObserver implementation.");
2576         } else {
2577           s = savedScanner;
2578         }
2579       }
2580       return s;
2581     } catch (Throwable t) {
2582       throw convertThrowableToIOE(cleanup(t, "Failed openScanner"));
2583     }
2584   }
2585 
2586   protected long addScanner(RegionScanner s) throws LeaseStillHeldException {
2587     long scannerId = -1L;
2588     scannerId = rand.nextLong();
2589     String scannerName = String.valueOf(scannerId);
2590     scanners.put(scannerName, s);
2591     this.leases.createLease(scannerName, new ScannerListener(scannerName));
2592     return scannerId;
2593   }
2594 
2595   public Result next(final long scannerId) throws IOException {
2596     Result[] res = next(scannerId, 1);
2597     if (res == null || res.length == 0) {
2598       return null;
2599     }
2600     return res[0];
2601   }
2602 
2603   public Result[] next(final long scannerId, int nbRows) throws IOException {
2604     String scannerName = String.valueOf(scannerId);
2605     RegionScanner s = this.scanners.get(scannerName);
2606     if (s == null) {
2607       LOG.info("Client tried to access missing scanner " + scannerName);
2608       throw new UnknownScannerException("Name: " + scannerName);
2609     }
2610     return internalNext(s, nbRows, scannerName);
2611   }
2612 
2613   private Result[] internalNext(final RegionScanner s, int nbRows,
2614       String scannerName) throws IOException {
2615     try {
2616       checkOpen();
2617     } catch (IOException e) {
2618       // If checkOpen failed, server not running or filesystem gone,
2619       // cancel this lease; filesystem is gone or we're closing or something.
2620       if (scannerName != null) {
2621         try {
2622           this.leases.cancelLease(scannerName);
2623         } catch (LeaseException le) {
2624           LOG.info("Server shutting down and client tried to access missing scanner "
2625               + scannerName);
2626         }
2627       }
2628       throw e;
2629     }
2630     Leases.Lease lease = null;
2631     try {
2632       // Remove lease while its being processed in server; protects against case
2633       // where processing of request takes > lease expiration time.
2634       try {
2635         if (scannerName != null) {
2636           lease = this.leases.removeLease(scannerName);
2637         }
2638       } catch (LeaseException le) {
2639         // What it really means is that there's no such scanner.
2640         LOG.info("Client tried to access missing scanner " + scannerName + " (no lease)");
2641         throw new UnknownScannerException("No lease for " + scannerName + ": " + le.getMessage());
2642       }
2643       List<Result> results = new ArrayList<Result>(nbRows);
2644       long currentScanResultSize = 0;
2645       List<KeyValue> values = new ArrayList<KeyValue>();
2646 
2647       // Call coprocessor. Get region info from scanner.
2648       HRegion region = getRegion(s.getRegionInfo().getRegionName());
2649       if (region != null && region.getCoprocessorHost() != null) {
2650         Boolean bypass = region.getCoprocessorHost().preScannerNext(s,
2651             results, nbRows);
2652         if (!results.isEmpty()) {
2653           for (Result r : results) {
2654             if (maxScannerResultSize < Long.MAX_VALUE){
2655               for (KeyValue kv : r.raw()) {
2656                 currentScanResultSize += kv.heapSize();
2657               }
2658             }
2659           }
2660         }
2661         if (bypass != null) {
2662           return s.isFilterDone() && results.isEmpty() ? null
2663               : results.toArray(new Result[0]);
2664         }
2665       }
2666 
2667       MultiVersionConsistencyControl.setThreadReadPoint(s.getMvccReadPoint());
2668       region.startRegionOperation();
2669       try {
2670         int i = 0;
2671         synchronized(s) {
2672           for (; i < nbRows
2673               && currentScanResultSize < maxScannerResultSize; ) {
2674             // Collect values to be returned here
2675             boolean moreRows = s.nextRaw(values, SchemaMetrics.METRIC_NEXTSIZE);
2676             if (!values.isEmpty()) {
2677               if (maxScannerResultSize < Long.MAX_VALUE){
2678                 for (KeyValue kv : values) {
2679                   currentScanResultSize += kv.heapSize();
2680                 }
2681               }
2682               results.add(new Result(values));
2683               i++;
2684             }
2685             if (!moreRows) {
2686               break;
2687             }
2688             values.clear();
2689           }
2690         }
2691         requestCount.addAndGet(i);
2692         region.readRequestsCount.add(i);
2693         region.setOpMetricsReadRequestCount(region.readRequestsCount.get());
2694       } finally {
2695         region.closeRegionOperation();
2696       }
2697       // coprocessor postNext hook
2698       if (region != null && region.getCoprocessorHost() != null) {
2699         region.getCoprocessorHost().postScannerNext(s, results, nbRows, true);
2700       }
2701 
2702       // If the scanner's filter - if any - is done with the scan
2703       // and wants to tell the client to stop the scan. This is done by passing
2704       // a null result.
2705       return s.isFilterDone() && results.isEmpty() ? null
2706           : results.toArray(new Result[0]);
2707     } catch (Throwable t) {
2708       if (t instanceof NotServingRegionException && scannerName != null) {
2709         this.scanners.remove(scannerName);
2710       }
2711       throw convertThrowableToIOE(cleanup(t));
2712     } finally {
2713       // We're done. On way out readd the above removed lease.  Adding resets
2714       // expiration time on lease.
2715       if (scannerName != null && this.scanners.containsKey(scannerName)) {
2716         if (lease != null) this.leases.addLease(lease);
2717       }
2718     }
2719   }
2720 
2721   public void close(final long scannerId) throws IOException {
2722     String scannerName = String.valueOf(scannerId);
2723     RegionScanner s = scanners.get(scannerName);
2724     internalCloseScanner(s, scannerName);
2725   }
2726 
2727   private void internalCloseScanner(final RegionScanner s, String scannerName)
2728       throws IOException {
2729     try {
2730       checkOpen();
2731       requestCount.incrementAndGet();
2732 
2733       HRegion region = null;
2734       if (s != null) {
2735         // call coprocessor.
2736         region = getRegion(s.getRegionInfo().getRegionName());
2737         if (region != null && region.getCoprocessorHost() != null) {
2738           if (region.getCoprocessorHost().preScannerClose(s)) {
2739             return; // bypass
2740           }
2741         }
2742       }
2743       RegionScanner toCloseScanner = s;
2744       if (scannerName != null) {
2745         toCloseScanner = scanners.remove(scannerName);
2746       }
2747       if (toCloseScanner != null) {
2748         toCloseScanner.close();
2749         if (scannerName != null) {
2750           this.leases.cancelLease(scannerName);
2751         }
2752 
2753         if (region != null && region.getCoprocessorHost() != null) {
2754           region.getCoprocessorHost().postScannerClose(toCloseScanner);
2755         }
2756       }
2757     } catch (Throwable t) {
2758       throw convertThrowableToIOE(cleanup(t));
2759     }
2760   }
2761 
2762   @Override
2763   public Result[] scan(byte[] regionName, Scan scan, int numberOfRows)
2764       throws IOException {
2765     RegionScanner s = internalOpenScanner(regionName, scan);
2766     try {
2767       Result[] results = internalNext(s, numberOfRows, null);
2768       return results;
2769     } finally {
2770       internalCloseScanner(s, null);
2771     }
2772   }
2773 
2774   /**
2775    * Instantiated as a scanner lease. If the lease times out, the scanner is
2776    * closed
2777    */
2778   private class ScannerListener implements LeaseListener {
2779     private final String scannerName;
2780 
2781     ScannerListener(final String n) {
2782       this.scannerName = n;
2783     }
2784 
2785     public void leaseExpired() {
2786       RegionScanner s = scanners.remove(this.scannerName);
2787       if (s != null) {
2788         LOG.info("Scanner " + this.scannerName + " lease expired on region "
2789             + s.getRegionInfo().getRegionNameAsString());
2790         try {
2791           HRegion region = getRegion(s.getRegionInfo().getRegionName());
2792           if (region != null && region.getCoprocessorHost() != null) {
2793             region.getCoprocessorHost().preScannerClose(s);
2794           }
2795 
2796           s.close();
2797           if (region != null && region.getCoprocessorHost() != null) {
2798             region.getCoprocessorHost().postScannerClose(s);
2799           }
2800         } catch (IOException e) {
2801           LOG.error("Closing scanner for "
2802               + s.getRegionInfo().getRegionNameAsString(), e);
2803         }
2804       } else {
2805         LOG.info("Scanner " + this.scannerName + " lease expired");
2806       }
2807     }
2808   }
2809 
2810   //
2811   // Methods that do the actual work for the remote API
2812   //
2813   public void delete(final byte[] regionName, final Delete delete)
2814       throws IOException {
2815     checkOpen();
2816     try {
2817       boolean writeToWAL = delete.getWriteToWAL();
2818       this.requestCount.incrementAndGet();
2819       HRegion region = getRegion(regionName);
2820       if (!region.getRegionInfo().isMetaTable()) {
2821         this.cacheFlusher.reclaimMemStoreMemory();
2822       }
2823       Integer lid = getLockFromId(delete.getLockId());
2824       region.delete(delete, lid, writeToWAL);
2825     } catch (Throwable t) {
2826       throw convertThrowableToIOE(cleanup(t));
2827     }
2828   }
2829 
2830   public int delete(final byte[] regionName, final List<Delete> deletes)
2831       throws IOException {
2832     checkOpen();
2833     // Count of Deletes processed.
2834     int i = 0;
2835     HRegion region = null;
2836     try {
2837       region = getRegion(regionName);
2838       if (!region.getRegionInfo().isMetaTable()) {
2839         this.cacheFlusher.reclaimMemStoreMemory();
2840       }
2841       int size = deletes.size();
2842       Integer[] locks = new Integer[size];
2843       for (Delete delete : deletes) {
2844         this.requestCount.incrementAndGet();
2845         locks[i] = getLockFromId(delete.getLockId());
2846         region.delete(delete, locks[i], delete.getWriteToWAL());
2847         i++;
2848       }
2849     } catch (WrongRegionException ex) {
2850       LOG.debug("Batch deletes: " + i, ex);
2851       return i;
2852     } catch (NotServingRegionException ex) {
2853       return i;
2854     } catch (Throwable t) {
2855       throw convertThrowableToIOE(cleanup(t));
2856     }
2857     return -1;
2858   }
2859 
2860   /**
2861    * @deprecated {@link RowLock} and associated operations are deprecated.
2862    */
2863   public long lockRow(byte[] regionName, byte[] row) throws IOException {
2864     checkOpen();
2865     NullPointerException npe = null;
2866     if (regionName == null) {
2867       npe = new NullPointerException("regionName is null");
2868     } else if (row == null) {
2869       npe = new NullPointerException("row to lock is null");
2870     }
2871     if (npe != null) {
2872       IOException io = new IOException("Invalid arguments to lockRow");
2873       io.initCause(npe);
2874       throw io;
2875     }
2876     requestCount.incrementAndGet();
2877     try {
2878       HRegion region = getRegion(regionName);
2879       if (region.getCoprocessorHost() != null) {
2880         region.getCoprocessorHost().preLockRow(regionName, row);
2881       }
2882       Integer r = region.obtainRowLock(row);
2883       long lockId = addRowLock(r, region);
2884       LOG.debug("Row lock " + lockId + " explicitly acquired by client");
2885       return lockId;
2886     } catch (Throwable t) {
2887       throw convertThrowableToIOE(cleanup(t, "Error obtaining row lock (fsOk: "
2888           + this.fsOk + ")"));
2889     }
2890   }
2891 
2892   protected long addRowLock(Integer r, HRegion region)
2893       throws LeaseStillHeldException {
2894     long lockId = -1L;
2895     lockId = rand.nextLong();
2896     String lockName = String.valueOf(lockId);
2897     rowlocks.put(lockName, r);
2898     this.leases.createLease(lockName, new RowLockListener(lockName, region));
2899     return lockId;
2900   }
2901 
2902   /**
2903    * Method to get the Integer lock identifier used internally from the long
2904    * lock identifier used by the client.
2905    *
2906    * @param lockId
2907    *          long row lock identifier from client
2908    * @return intId Integer row lock used internally in HRegion
2909    * @throws IOException
2910    *           Thrown if this is not a valid client lock id.
2911    */
2912   Integer getLockFromId(long lockId) throws IOException {
2913     if (lockId == -1L) {
2914       return null;
2915     }
2916     String lockName = String.valueOf(lockId);
2917     Integer rl = rowlocks.get(lockName);
2918     if (rl == null) {
2919       throw new UnknownRowLockException("Invalid row lock");
2920     }
2921     this.leases.renewLease(lockName);
2922     return rl;
2923   }
2924 
2925   /**
2926    * @deprecated {@link RowLock} and associated operations are deprecated.
2927    */
2928   @Override
2929   @QosPriority(priority=HConstants.HIGH_QOS)
2930   public void unlockRow(byte[] regionName, long lockId) throws IOException {
2931     checkOpen();
2932     NullPointerException npe = null;
2933     if (regionName == null) {
2934       npe = new NullPointerException("regionName is null");
2935     } else if (lockId == -1L) {
2936       npe = new NullPointerException("lockId is null");
2937     }
2938     if (npe != null) {
2939       IOException io = new IOException("Invalid arguments to unlockRow");
2940       io.initCause(npe);
2941       throw io;
2942     }
2943     requestCount.incrementAndGet();
2944     try {
2945       HRegion region = getRegion(regionName);
2946       if (region.getCoprocessorHost() != null) {
2947         region.getCoprocessorHost().preUnLockRow(regionName, lockId);
2948       }
2949       String lockName = String.valueOf(lockId);
2950       Integer r = rowlocks.remove(lockName);
2951       if (r == null) {
2952         throw new UnknownRowLockException(lockName);
2953       }
2954       region.releaseRowLock(r);
2955       this.leases.cancelLease(lockName);
2956       LOG.debug("Row lock " + lockId
2957           + " has been explicitly released by client");
2958     } catch (Throwable t) {
2959       throw convertThrowableToIOE(cleanup(t));
2960     }
2961   }
2962 
2963   /**
2964    * Atomically bulk load several HFiles into an open region
2965    * @return true if successful, false is failed but recoverably (no action)
2966    * @throws IOException if failed unrecoverably
2967    */
2968   @Override
2969   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
2970       byte[] regionName) throws IOException {
2971     return bulkLoadHFiles(familyPaths, regionName, false);
2972   }
2973 
2974   /**
2975    * Atomically bulk load several HFiles into an open region
2976    * @return true if successful, false is failed but recoverably (no action)
2977    * @throws IOException if failed unrecoverably
2978    */
2979   @Override
2980   public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
2981       byte[] regionName, boolean assignSeqNum) throws IOException {
2982     checkOpen();
2983     HRegion region = getRegion(regionName);
2984     boolean bypass = false;
2985     if (region.getCoprocessorHost() != null) {
2986       bypass = region.getCoprocessorHost().preBulkLoadHFile(familyPaths);
2987     }
2988     boolean loaded = false;
2989     if (!bypass) {
2990       loaded = region.bulkLoadHFiles(familyPaths, assignSeqNum);
2991     }
2992     if (region.getCoprocessorHost() != null) {
2993       loaded = region.getCoprocessorHost().postBulkLoadHFile(familyPaths, loaded);
2994     }
2995     return loaded;
2996   }
2997 
2998   Map<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
2999 
3000   /**
3001    * Instantiated as a row lock lease. If the lease times out, the row lock is
3002    * released
3003    */
3004   private class RowLockListener implements LeaseListener {
3005     private final String lockName;
3006     private final HRegion region;
3007 
3008     RowLockListener(final String lockName, final HRegion region) {
3009       this.lockName = lockName;
3010       this.region = region;
3011     }
3012 
3013     public void leaseExpired() {
3014       LOG.info("Row Lock " + this.lockName + " lease expired");
3015       Integer r = rowlocks.remove(this.lockName);
3016       if (r != null) {
3017         region.releaseRowLock(r);
3018       }
3019     }
3020   }
3021 
3022   // Region open/close direct RPCs
3023 
3024   @Override
3025   @QosPriority(priority=HConstants.HIGH_QOS)
3026   public RegionOpeningState openRegion(HRegionInfo region)
3027   throws IOException {
3028     return openRegion(region, -1);
3029   }
3030 
3031   @Override
3032   @QosPriority(priority = HConstants.HIGH_QOS)
3033   public RegionOpeningState openRegion(HRegionInfo region, int versionOfOfflineNode)
3034       throws IOException {
3035     return openRegion(region, versionOfOfflineNode, null);
3036   }
3037 
3038   private RegionOpeningState openRegion(HRegionInfo region, int versionOfOfflineNode,
3039       Map<String, HTableDescriptor> htds) throws IOException {
3040     checkOpen();
3041     HRegion onlineRegion = this.getFromOnlineRegions(region.getEncodedName());
3042     if (null != onlineRegion) {
3043       // See HBASE-5094. Cross check with META if still this RS is owning the
3044       // region.
3045       Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(
3046           this.catalogTracker, region.getRegionName());
3047       if (this.getServerName().equals(p.getSecond())) {
3048         LOG.warn("Attempted open of " + region.getEncodedName()
3049             + " but already online on this server");
3050         return RegionOpeningState.ALREADY_OPENED;
3051       } else {
3052         LOG.warn("The region " + region.getEncodedName()
3053             + " is online on this server but META does not have this server.");
3054         this.removeFromOnlineRegions(region.getEncodedName());
3055       }
3056     }
3057     // Added to in-memory RS RIT that we are trying to open this region.
3058     // Clear it if we fail queuing an open executor.
3059     boolean isNewRit = addRegionsInTransition(region, OPEN);
3060     if (!isNewRit) {
3061       // An open is in progress. This is supported, but let's log this.
3062       LOG.info("Receiving OPEN for the region:" +
3063           region.getRegionNameAsString() + " , which we are already trying to OPEN" +
3064           " - ignoring this new request for this region.");
3065       return RegionOpeningState.OPENED;
3066     }
3067     try {
3068       LOG.info("Received request to open region: " +
3069         region.getRegionNameAsString());
3070       HTableDescriptor htd = null;
3071       if (htds == null) {
3072         htd = this.tableDescriptors.get(region.getTableName());
3073       } else {
3074         htd = htds.get(region.getTableNameAsString());
3075         if (htd == null) {
3076           htd = this.tableDescriptors.get(region.getTableName());
3077           htds.put(region.getTableNameAsString(), htd);
3078         }
3079       }
3080 
3081       // Mark the region as OPENING up in zk.  This is how we tell the master control of the
3082       // region has passed to this regionserver.
3083       int version = transitionZookeeperOfflineToOpening(region, versionOfOfflineNode);
3084       // Need to pass the expected version in the constructor.
3085       if (region.isRootRegion()) {
3086         this.service.submit(new OpenRootHandler(this, this, region, htd, version));
3087       } else if (region.isMetaRegion()) {
3088         this.service.submit(new OpenMetaHandler(this, this, region, htd, version));
3089       } else {
3090         this.service.submit(new OpenRegionHandler(this, this, region, htd, version));
3091       }
3092     } catch (IOException ie) {
3093       // Clear from this server's RIT list else will stick around for ever.
3094       removeFromRegionsInTransition(region);
3095       throw ie;
3096     }
3097     return RegionOpeningState.OPENED;
3098   }
3099 
3100   /**
3101    * Transition ZK node from OFFLINE to OPENING. The master will get a callback
3102    * and will know that the region is now ours.
3103    *
3104    * @param hri
3105    *          HRegionInfo whose znode we are updating
3106    * @param versionOfOfflineNode
3107    *          Version Of OfflineNode that needs to be compared before changing
3108    *          the node's state from OFFLINE
3109    * @throws IOException
3110    */
3111   int transitionZookeeperOfflineToOpening(final HRegionInfo hri, int versionOfOfflineNode)
3112       throws IOException {
3113     // TODO: should also handle transition from CLOSED?
3114     int version = -1;
3115     try {
3116       // Initialize the znode version.
3117       version = ZKAssign.transitionNode(this.zooKeeper, hri, this.getServerName(),
3118           EventType.M_ZK_REGION_OFFLINE, EventType.RS_ZK_REGION_OPENING, versionOfOfflineNode);
3119     } catch (KeeperException e) {
3120       LOG.error("Error transition from OFFLINE to OPENING for region=" + hri.getEncodedName(), e);
3121     }
3122     if (version == -1) {
3123       // TODO: Fix this sloppyness. The exception should be coming off zk
3124       // directly, not an
3125       // intepretation at this high-level (-1 when we call transitionNode can
3126       // mean many things).
3127       throw new IOException("Failed transition from OFFLINE to OPENING for region="
3128           + hri.getEncodedName());
3129     }
3130     return version;
3131   }
3132 
3133    /**
3134     * String currentAction) throws RegionAlreadyInTransitionException { Add
3135     * region to this regionservers list of in transitions regions ONLY if its not
3136     * already byte[] encodedName = region.getEncodedNameAsBytes(); in transition.
3137     * If a region already in RIT, we throw
3138     * {@link RegionAlreadyInTransitionException}. if
3139     * (this.regionsInTransitionInRS.containsKey(encodedName)) { Callers need to
3140     * call {@link #removeFromRegionsInTransition(HRegionInfo)} when done or if
3141     * boolean openAction = this.regionsInTransitionInRS.get(encodedName); error
3142     * processing.
3143     *
3144     * @param region
3145     *          Region to add
3146     * @param currentAction
3147     *          Whether OPEN or CLOSE.
3148     * @throws RegionAlreadyInTransitionException
3149     */
3150    protected boolean addRegionsInTransition(final HRegionInfo region, final String currentAction)
3151        throws RegionAlreadyInTransitionException {
3152      boolean isOpen = currentAction.equals(OPEN);
3153      Boolean action = this.regionsInTransitionInRS.putIfAbsent(
3154          region.getEncodedNameAsBytes(), isOpen);
3155      if (action == null) return true;
3156      if (isOpen && action.booleanValue()) {
3157        return false;
3158      }
3159      // The below exception message will be used in master.
3160      throw new RegionAlreadyInTransitionException("Received:" + currentAction
3161          + " for the region:" + region.getRegionNameAsString()
3162          + ", which we are already trying to " + (action ? OPEN : CLOSE) + ".");
3163    }
3164 
3165   @Override
3166   @QosPriority(priority=HConstants.HIGH_QOS)
3167   public void openRegions(List<HRegionInfo> regions)
3168   throws IOException {
3169     checkOpen();
3170     LOG.info("Received request to open " + regions.size() + " region(s)");
3171     Map<String, HTableDescriptor> htds = new HashMap<String, HTableDescriptor>(regions.size());
3172     for (HRegionInfo region : regions) openRegion(region, -1, htds);
3173   }
3174 
3175   @Override
3176   @QosPriority(priority=HConstants.HIGH_QOS)
3177   public boolean closeRegion(HRegionInfo region)
3178   throws IOException {
3179     return closeRegion(region, true, -1);
3180   }
3181 
3182   @Override
3183   @QosPriority(priority=HConstants.HIGH_QOS)
3184   public boolean closeRegion(final HRegionInfo region,
3185     final int versionOfClosingNode)
3186   throws IOException {
3187     return closeRegion(region, true, versionOfClosingNode);
3188   }
3189 
3190   @Override
3191   @QosPriority(priority=HConstants.HIGH_QOS)
3192   public boolean closeRegion(HRegionInfo region, final boolean zk)
3193   throws IOException {
3194     return closeRegion(region, zk, -1);
3195   }
3196 
3197   @QosPriority(priority=HConstants.HIGH_QOS)
3198   protected boolean closeRegion(HRegionInfo region, final boolean zk,
3199     final int versionOfClosingNode)
3200   throws IOException {
3201     checkOpen();
3202     //Check for permissions to close.
3203     HRegion actualRegion = this.getFromOnlineRegions(region.getEncodedName());
3204     if (actualRegion != null && actualRegion.getCoprocessorHost() != null) {
3205       actualRegion.getCoprocessorHost().preClose(false);
3206     }
3207     LOG.info("Received close region: " + region.getRegionNameAsString() +
3208       ". Version of ZK closing node:" + versionOfClosingNode);
3209     boolean hasit = this.onlineRegions.containsKey(region.getEncodedName());
3210     if (!hasit) {
3211       LOG.warn("Received close for region we are not serving; " +
3212         region.getEncodedName());
3213       throw new NotServingRegionException("Received close for "
3214         + region.getRegionNameAsString() + " but we are not serving it");
3215     }
3216     return closeRegion(region, false, zk, versionOfClosingNode);
3217   }
3218 
3219   @Override
3220   @QosPriority(priority=HConstants.HIGH_QOS)
3221   public boolean closeRegion(byte[] encodedRegionName, boolean zk)
3222     throws IOException {
3223     return closeRegion(encodedRegionName, false, zk);
3224   }
3225 
3226   /**
3227    * @param region Region to close
3228    * @param abort True if we are aborting
3229    * @param zk True if we are to update zk about the region close; if the close
3230    * was orchestrated by master, then update zk.  If the close is being run by
3231    * the regionserver because its going down, don't update zk.
3232    * @return True if closed a region.
3233    */
3234   protected boolean closeRegion(HRegionInfo region, final boolean abort,
3235       final boolean zk) {
3236     return closeRegion(region, abort, zk, -1);
3237   }
3238 
3239 
3240   /**
3241    * @param region Region to close
3242    * @param abort True if we are aborting
3243    * @param zk True if we are to update zk about the region close; if the close
3244    * was orchestrated by master, then update zk.  If the close is being run by
3245    * the regionserver because its going down, don't update zk.
3246    * @param versionOfClosingNode
3247    *   the version of znode to compare when RS transitions the znode from
3248    *   CLOSING state.
3249    * @return True if closed a region.
3250    */
3251   protected boolean closeRegion(HRegionInfo region, final boolean abort,
3252       final boolean zk, final int versionOfClosingNode) {
3253     
3254     HRegion actualRegion = this.getFromOnlineRegions(region.getEncodedName());
3255     if (actualRegion != null && actualRegion.getCoprocessorHost() != null) {
3256       try {
3257         actualRegion.getCoprocessorHost().preClose(abort);
3258       } catch (IOException e) {
3259         LOG.warn(e);
3260         return false;
3261       }
3262     }
3263     try {
3264       addRegionsInTransition(region, CLOSE);
3265     } catch (RegionAlreadyInTransitionException rate) {
3266       LOG.warn("Received close for region we are already opening or closing; "
3267           + region.getEncodedName());
3268       return false;
3269     }
3270     boolean success = false;
3271     try {
3272       CloseRegionHandler crh = null;
3273       if (region.isRootRegion()) {
3274         crh = new CloseRootHandler(this, this, region, abort, zk, versionOfClosingNode);
3275       } else if (region.isMetaRegion()) {
3276         crh = new CloseMetaHandler(this, this, region, abort, zk, versionOfClosingNode);
3277       } else {
3278         crh = new CloseRegionHandler(this, this, region, abort, zk, versionOfClosingNode);
3279       }
3280       this.service.submit(crh);
3281       success = true;
3282     } finally {
3283       // Remove from this server's RIT.
3284       if (!success) removeFromRegionsInTransition(region);
3285     }
3286     return true;
3287   }
3288 
3289   /**
3290    * @param encodedRegionName
3291    *          encodedregionName to close
3292    * @param abort
3293    *          True if we are aborting
3294    * @param zk
3295    *          True if we are to update zk about the region close; if the close
3296    *          was orchestrated by master, then update zk. If the close is being
3297    *          run by the regionserver because its going down, don't update zk.
3298    * @return True if closed a region.
3299    */
3300   protected boolean closeRegion(byte[] encodedRegionName, final boolean abort,
3301       final boolean zk) throws IOException {
3302     String encodedRegionNameStr = Bytes.toString(encodedRegionName);
3303     HRegion region = this.getFromOnlineRegions(encodedRegionNameStr);
3304     if (null != region) {
3305       return closeRegion(region.getRegionInfo(), abort, zk);
3306     }
3307     LOG.error("The specified region name" + encodedRegionNameStr
3308         + " does not exist to close the region.");
3309     return false;
3310   }
3311 
3312   // Manual remote region administration RPCs
3313 
3314   @Override
3315   @QosPriority(priority=HConstants.HIGH_QOS)
3316   public void flushRegion(HRegionInfo regionInfo)
3317       throws NotServingRegionException, IOException {
3318     checkOpen();
3319     LOG.info("Flushing " + regionInfo.getRegionNameAsString());
3320     HRegion region = getRegion(regionInfo.getRegionName());
3321     boolean needsCompaction = region.flushcache();
3322     if (needsCompaction) {
3323       this.compactSplitThread.requestCompaction(region, "Compaction through user triggered flush");
3324     }
3325   }
3326 
3327   @Override
3328   @QosPriority(priority=HConstants.HIGH_QOS)
3329   public void splitRegion(HRegionInfo regionInfo)
3330       throws NotServingRegionException, IOException {
3331     splitRegion(regionInfo, null);
3332   }
3333 
3334   @Override
3335   public void splitRegion(HRegionInfo regionInfo, byte[] splitPoint)
3336       throws NotServingRegionException, IOException {
3337     checkOpen();
3338     HRegion region = getRegion(regionInfo.getRegionName());
3339     region.flushcache();
3340     region.forceSplit(splitPoint);
3341     compactSplitThread.requestSplit(region, region.checkSplit());
3342   }
3343 
3344   @Override
3345   @QosPriority(priority=HConstants.HIGH_QOS)
3346   public void compactRegion(HRegionInfo regionInfo, boolean major)
3347       throws NotServingRegionException, IOException {
3348     compactRegion(regionInfo, major, null);
3349   }
3350 
3351   @Override
3352   @QosPriority(priority=HConstants.HIGH_QOS)
3353   public void compactRegion(HRegionInfo regionInfo, boolean major,  byte[] family)
3354       throws NotServingRegionException, IOException {
3355     checkOpen();
3356     HRegion region = getRegion(regionInfo.getRegionName());
3357     Store store = null;
3358     if (family != null) {
3359       store = region.getStore(family);
3360       if (store == null) {
3361         throw new IOException("column family " + Bytes.toString(family) +
3362           " does not exist in region " + new String(region.getRegionNameAsString()));
3363       }
3364     }
3365 
3366     if (major) {
3367       if (family != null) {
3368         store.triggerMajorCompaction();
3369       } else {
3370         region.triggerMajorCompaction();
3371       }
3372     }
3373     String familyLogMsg = (family != null)?" for column family: " + Bytes.toString(family):"";
3374     LOG.trace("User-triggered compaction requested for region " +
3375       region.getRegionNameAsString() + familyLogMsg);
3376     String log = "User-triggered " + (major ? "major " : "") + "compaction" + familyLogMsg;
3377     if (family != null) {
3378       compactSplitThread.requestCompaction(region, store, log,
3379         Store.PRIORITY_USER, null);
3380     } else {
3381       compactSplitThread.requestCompaction(region, log,
3382         Store.PRIORITY_USER, null);
3383     }
3384   }
3385 
3386   /** @return the info server */
3387   public InfoServer getInfoServer() {
3388     return infoServer;
3389   }
3390 
3391   /**
3392    * @return true if a stop has been requested.
3393    */
3394   public boolean isStopped() {
3395     return this.stopped;
3396   }
3397 
3398   @Override
3399   public boolean isStopping() {
3400     return this.stopping;
3401   }
3402 
3403   /**
3404    *
3405    * @return the configuration
3406    */
3407   public Configuration getConfiguration() {
3408     return conf;
3409   }
3410 
3411   /** @return the write lock for the server */
3412   ReentrantReadWriteLock.WriteLock getWriteLock() {
3413     return lock.writeLock();
3414   }
3415 
3416   @Override
3417   @QosPriority(priority=HConstants.HIGH_QOS)
3418   public List<HRegionInfo> getOnlineRegions() throws IOException {
3419     checkOpen();
3420     List<HRegionInfo> list = new ArrayList<HRegionInfo>(onlineRegions.size());
3421     for (Map.Entry<String,HRegion> e: this.onlineRegions.entrySet()) {
3422       list.add(e.getValue().getRegionInfo());
3423     }
3424     Collections.sort(list);
3425     return list;
3426   }
3427 
3428   public int getNumberOfOnlineRegions() {
3429     return this.onlineRegions.size();
3430   }
3431 
3432   boolean isOnlineRegionsEmpty() {
3433     return this.onlineRegions.isEmpty();
3434   }
3435 
3436   /**
3437    * @param encodedRegionName
3438    * @return JSON Map of labels to values for passed in <code>encodedRegionName</code>
3439    * @throws IOException
3440    */
3441   public byte [] getRegionStats(final String encodedRegionName)
3442   throws IOException {
3443     HRegion r = null;
3444     synchronized (this.onlineRegions) {
3445       r = this.onlineRegions.get(encodedRegionName);
3446     }
3447     if (r == null) return null;
3448     ObjectMapper mapper = new ObjectMapper();
3449     int stores = 0;
3450     int storefiles = 0;
3451     int storefileSizeMB = 0;
3452     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
3453     int storefileIndexSizeMB = 0;
3454     long totalCompactingKVs = 0;
3455     long currentCompactedKVs = 0;
3456     synchronized (r.stores) {
3457       stores += r.stores.size();
3458       for (Store store : r.stores.values()) {
3459         storefiles += store.getStorefilesCount();
3460         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
3461         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
3462       }
3463     }
3464     Map<String, Integer> map = new TreeMap<String, Integer>();
3465     map.put("stores", stores);
3466     map.put("storefiles", storefiles);
3467     map.put("storefileSizeMB", storefileIndexSizeMB);
3468     map.put("memstoreSizeMB", memstoreSizeMB);
3469     StringWriter w = new StringWriter();
3470     mapper.writeValue(w, map);
3471     w.close();
3472     return Bytes.toBytes(w.toString());
3473   }
3474 
3475   /**
3476    * For tests and web ui.
3477    * This method will only work if HRegionServer is in the same JVM as client;
3478    * HRegion cannot be serialized to cross an rpc.
3479    * @see #getOnlineRegions()
3480    */
3481   public Collection<HRegion> getOnlineRegionsLocalContext() {
3482     Collection<HRegion> regions = this.onlineRegions.values();
3483     return Collections.unmodifiableCollection(regions);
3484   }
3485 
3486   @Override
3487   public void addToOnlineRegions(HRegion region) {
3488     this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
3489   }
3490 
3491   @Override
3492   public boolean removeFromOnlineRegions(final String encodedName) {
3493     HRegion toReturn = null;
3494     toReturn = this.onlineRegions.remove(encodedName);
3495     
3496     //Clear all of the dynamic metrics as they are now probably useless.
3497     //This is a clear because dynamic metrics could include metrics per cf and
3498     //per hfile.  Figuring out which cfs, hfiles, and regions are still relevant to
3499     //this region server would be an onerous task.  Instead just clear everything
3500     //and on the next tick of the metrics everything that is still relevant will be
3501     //re-added.
3502     this.dynamicMetrics.clear();
3503     return toReturn != null;
3504   }
3505 
3506   /**
3507    * @return A new Map of online regions sorted by region size with the first
3508    *         entry being the biggest.
3509    */
3510   public SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
3511     // we'll sort the regions in reverse
3512     SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
3513         new Comparator<Long>() {
3514           public int compare(Long a, Long b) {
3515             return -1 * a.compareTo(b);
3516           }
3517         });
3518     // Copy over all regions. Regions are sorted by size with biggest first.
3519     for (HRegion region : this.onlineRegions.values()) {
3520       sortedRegions.put(Long.valueOf(region.memstoreSize.get()), region);
3521     }
3522     return sortedRegions;
3523   }
3524 
3525   @Override
3526   public HRegion getFromOnlineRegions(final String encodedRegionName) {
3527     HRegion r = null;
3528     r = this.onlineRegions.get(encodedRegionName);
3529     return r;
3530   }
3531 
3532   /**
3533    * @param regionName
3534    * @return HRegion for the passed binary <code>regionName</code> or null if
3535    *         named region is not member of the online regions.
3536    */
3537   public HRegion getOnlineRegion(final byte[] regionName) {
3538     return getFromOnlineRegions(HRegionInfo.encodeRegionName(regionName));
3539   }
3540 
3541   /** @return the request count */
3542   public AtomicInteger getRequestCount() {
3543     return this.requestCount;
3544   }
3545 
3546   /**
3547    * @return time stamp in millis of when this region server was started
3548    */
3549   public long getStartcode() {
3550     return this.startcode;
3551   }
3552 
3553   /** @return reference to FlushRequester */
3554   public FlushRequester getFlushRequester() {
3555     return this.cacheFlusher;
3556   }
3557 
3558   /**
3559    * Protected utility method for safely obtaining an HRegion handle.
3560    *
3561    * @param regionName
3562    *          Name of online {@link HRegion} to return
3563    * @return {@link HRegion} for <code>regionName</code>
3564    * @throws NotServingRegionException
3565    */
3566   protected HRegion getRegion(final byte[] regionName)
3567       throws NotServingRegionException {
3568     HRegion region = null;
3569     region = getOnlineRegion(regionName);
3570     if (region == null) {
3571       throw new NotServingRegionException("Region is not online: " +
3572         Bytes.toStringBinary(regionName));
3573     }
3574     return region;
3575   }
3576 
3577   /**
3578    * Get the top N most loaded regions this server is serving so we can tell the
3579    * master which regions it can reallocate if we're overloaded. TODO: actually
3580    * calculate which regions are most loaded. (Right now, we're just grabbing
3581    * the first N regions being served regardless of load.)
3582    */
3583   protected HRegionInfo[] getMostLoadedRegions() {
3584     ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
3585     for (HRegion r : onlineRegions.values()) {
3586       if (!r.isAvailable()) {
3587         continue;
3588       }
3589       if (regions.size() < numRegionsToReport) {
3590         regions.add(r.getRegionInfo());
3591       } else {
3592         break;
3593       }
3594     }
3595     return regions.toArray(new HRegionInfo[regions.size()]);
3596   }
3597 
3598   /**
3599    * Called to verify that this server is up and running.
3600    *
3601    * @throws IOException
3602    */
3603   protected void checkOpen() throws IOException {
3604     if (this.stopped || this.abortRequested) {
3605       throw new RegionServerStoppedException("Server " + getServerName() +
3606         " not running" + (this.abortRequested ? ", aborting" : ""));
3607     }
3608     if (!fsOk) {
3609       throw new RegionServerStoppedException("File system not available");
3610     }
3611   }
3612 
3613   @Override
3614   @QosPriority(priority=HConstants.HIGH_QOS)
3615   public ProtocolSignature getProtocolSignature(
3616       String protocol, long version, int clientMethodsHashCode)
3617   throws IOException {
3618     if (protocol.equals(HRegionInterface.class.getName())) {
3619       return new ProtocolSignature(HRegionInterface.VERSION, null);
3620     }
3621     throw new IOException("Unknown protocol: " + protocol);
3622   }
3623 
3624   @Override
3625   @QosPriority(priority=HConstants.HIGH_QOS)
3626   public long getProtocolVersion(final String protocol, final long clientVersion)
3627   throws IOException {
3628     if (protocol.equals(HRegionInterface.class.getName())) {
3629       return HRegionInterface.VERSION;
3630     }
3631     throw new IOException("Unknown protocol: " + protocol);
3632   }
3633 
3634   @Override
3635   public Leases getLeases() {
3636     return leases;
3637   }
3638 
3639   /**
3640    * @return Return the rootDir.
3641    */
3642   protected Path getRootDir() {
3643     return rootDir;
3644   }
3645 
3646   /**
3647    * @return Return the fs.
3648    */
3649   public FileSystem getFileSystem() {
3650     return fs;
3651   }
3652 
3653   /**
3654    * @return This servers {@link HServerInfo}
3655    */
3656   // TODO: Deprecate and do getServerName instead.
3657   public HServerInfo getServerInfo() {
3658     try {
3659       return getHServerInfo();
3660     } catch (IOException e) {
3661       e.printStackTrace();
3662     }
3663     return null;
3664   }
3665 
3666   @Override
3667   public void mutateRow(byte[] regionName, RowMutations rm)
3668       throws IOException {
3669     checkOpen();
3670     if (regionName == null) {
3671       throw new IOException("Invalid arguments to mutateRow " +
3672       "regionName is null");
3673     }
3674     requestCount.incrementAndGet();
3675     try {
3676       HRegion region = getRegion(regionName);
3677       if (!region.getRegionInfo().isMetaTable()) {
3678         this.cacheFlusher.reclaimMemStoreMemory();
3679       }
3680       region.mutateRow(rm);
3681     } catch (IOException e) {
3682       checkFileSystem();
3683       throw e;
3684     }
3685   }
3686 
3687   @Override
3688   public Result append(byte[] regionName, Append append)
3689   throws IOException {
3690     checkOpen();
3691     if (regionName == null) {
3692       throw new IOException("Invalid arguments to increment " +
3693       "regionName is null");
3694     }
3695     requestCount.incrementAndGet();
3696     try {
3697       HRegion region = getRegion(regionName);
3698       Integer lock = getLockFromId(append.getLockId());
3699       Append appVal = append;
3700       Result resVal;
3701       if (region.getCoprocessorHost() != null) {
3702         resVal = region.getCoprocessorHost().preAppend(appVal);
3703         if (resVal != null) {
3704           return resVal;
3705         }
3706       }
3707       resVal = region.append(appVal, lock, append.getWriteToWAL());
3708       if (region.getCoprocessorHost() != null) {
3709         region.getCoprocessorHost().postAppend(appVal, resVal);
3710       }
3711       return resVal;
3712     } catch (IOException e) {
3713       checkFileSystem();
3714       throw e;
3715     }
3716   }
3717 
3718   @Override
3719   public Result increment(byte[] regionName, Increment increment)
3720   throws IOException {
3721     checkOpen();
3722     if (regionName == null) {
3723       throw new IOException("Invalid arguments to increment " +
3724       "regionName is null");
3725     }
3726     requestCount.incrementAndGet();
3727     try {
3728       HRegion region = getRegion(regionName);
3729       Integer lock = getLockFromId(increment.getLockId());
3730       Increment incVal = increment;
3731       Result resVal;
3732       if (region.getCoprocessorHost() != null) {
3733         resVal = region.getCoprocessorHost().preIncrement(incVal);
3734         if (resVal != null) {
3735           return resVal;
3736         }
3737       }
3738       resVal = region.increment(incVal, lock,
3739           increment.getWriteToWAL());
3740       if (region.getCoprocessorHost() != null) {
3741         resVal = region.getCoprocessorHost().postIncrement(incVal, resVal);
3742       }
3743       return resVal;
3744     } catch (IOException e) {
3745       checkFileSystem();
3746       throw e;
3747     }
3748   }
3749 
3750   /** {@inheritDoc} */
3751   public long incrementColumnValue(byte[] regionName, byte[] row,
3752       byte[] family, byte[] qualifier, long amount, boolean writeToWAL)
3753       throws IOException {
3754     checkOpen();
3755 
3756     if (regionName == null) {
3757       throw new IOException("Invalid arguments to incrementColumnValue "
3758           + "regionName is null");
3759     }
3760     requestCount.incrementAndGet();
3761     try {
3762       HRegion region = getRegion(regionName);
3763       if (region.getCoprocessorHost() != null) {
3764         Long amountVal = region.getCoprocessorHost().preIncrementColumnValue(row,
3765           family, qualifier, amount, writeToWAL);
3766         if (amountVal != null) {
3767           return amountVal.longValue();
3768         }
3769       }
3770       long retval = region.incrementColumnValue(row, family, qualifier, amount,
3771         writeToWAL);
3772       if (region.getCoprocessorHost() != null) {
3773         retval = region.getCoprocessorHost().postIncrementColumnValue(row,
3774           family, qualifier, amount, writeToWAL, retval);
3775       }
3776       return retval;
3777     } catch (IOException e) {
3778       checkFileSystem();
3779       throw e;
3780     }
3781   }
3782 
3783   /** {@inheritDoc}
3784    * @deprecated Use {@link #getServerName()} instead.
3785    */
3786   @Override
3787   @QosPriority(priority=HConstants.HIGH_QOS)
3788   public HServerInfo getHServerInfo() throws IOException {
3789     checkOpen();
3790     return new HServerInfo(new HServerAddress(this.isa),
3791       this.startcode, this.webuiport);
3792   }
3793 
3794   @SuppressWarnings("unchecked")
3795   @Override
3796   public <R> MultiResponse multi(MultiAction<R> multi) throws IOException {
3797     checkOpen();
3798     MultiResponse response = new MultiResponse();
3799     for (Map.Entry<byte[], List<Action<R>>> e : multi.actions.entrySet()) {
3800       byte[] regionName = e.getKey();
3801       List<Action<R>> actionsForRegion = e.getValue();
3802       // sort based on the row id - this helps in the case where we reach the
3803       // end of a region, so that we don't have to try the rest of the
3804       // actions in the list.
3805       Collections.sort(actionsForRegion);
3806       Row action;
3807       List<Action<R>> mutations = new ArrayList<Action<R>>();
3808       for (Action<R> a : actionsForRegion) {
3809         action = a.getAction();
3810         int originalIndex = a.getOriginalIndex();
3811 
3812         try {
3813           if (action instanceof Delete || action instanceof Put) {
3814             mutations.add(a); 
3815           } else if (action instanceof Get) {
3816             response.add(regionName, originalIndex,
3817                 get(regionName, (Get)action));
3818           } else if (action instanceof Exec) {
3819             ExecResult result = execCoprocessor(regionName, (Exec)action);
3820             response.add(regionName, new Pair<Integer, Object>(
3821                 a.getOriginalIndex(), result.getValue()
3822             ));
3823           } else if (action instanceof Increment) {
3824             response.add(regionName, originalIndex,
3825                 increment(regionName, (Increment)action));
3826           } else if (action instanceof Append) {
3827             response.add(regionName, originalIndex,
3828                 append(regionName, (Append)action));
3829           } else if (action instanceof RowMutations) {
3830             mutateRow(regionName, (RowMutations)action);
3831             response.add(regionName, originalIndex, new Result());
3832           } else {
3833             LOG.debug("Error: invalid Action, row must be a Get, Delete, " +
3834                 "Put, Exec, Increment, or Append.");
3835             throw new DoNotRetryIOException("Invalid Action, row must be a " +
3836                 "Get, Delete, Put, Exec, Increment, or Append.");
3837           }
3838         } catch (IOException ex) {
3839           response.add(regionName, originalIndex, ex);
3840         }
3841       }
3842 
3843       // We do the puts with result.put so we can get the batching efficiency
3844       // we so need. All this data munging doesn't seem great, but at least
3845       // we arent copying bytes or anything.
3846       if (!mutations.isEmpty()) {
3847         try {
3848           HRegion region = getRegion(regionName);
3849 
3850           if (!region.getRegionInfo().isMetaTable()) {
3851             this.cacheFlusher.reclaimMemStoreMemory();
3852           }
3853 
3854           List<Pair<Mutation,Integer>> mutationsWithLocks =
3855               Lists.newArrayListWithCapacity(mutations.size());
3856           for (Action<R> a : mutations) {
3857             Mutation m = (Mutation) a.getAction();
3858 
3859             Integer lock;
3860             try {
3861               lock = getLockFromId(m.getLockId());
3862             } catch (UnknownRowLockException ex) {
3863               response.add(regionName, a.getOriginalIndex(), ex);
3864               continue;
3865             }
3866             mutationsWithLocks.add(new Pair<Mutation, Integer>(m, lock));
3867           }
3868 
3869           this.requestCount.addAndGet(mutations.size());
3870 
3871           OperationStatus[] codes =
3872               region.batchMutate(mutationsWithLocks.toArray(new Pair[]{}));
3873 
3874           for( int i = 0 ; i < codes.length ; i++) {
3875             OperationStatus code = codes[i];
3876 
3877             Action<R> theAction = mutations.get(i);
3878             Object result = null;
3879 
3880             if (code.getOperationStatusCode() == OperationStatusCode.SUCCESS) {
3881               result = new Result();
3882             } else if (code.getOperationStatusCode()
3883                 == OperationStatusCode.SANITY_CHECK_FAILURE) {
3884               // Don't send a FailedSanityCheckException as older clients will not know about
3885               // that class being a subclass of DoNotRetryIOException
3886               // and will retry mutations that will never succeed.
3887               result = new DoNotRetryIOException(code.getExceptionMsg());
3888             } else if (code.getOperationStatusCode() == OperationStatusCode.BAD_FAMILY) {
3889               result = new NoSuchColumnFamilyException(code.getExceptionMsg());
3890             }
3891             // FAILURE && NOT_RUN becomes null, aka: need to run again.
3892 
3893             response.add(regionName, theAction.getOriginalIndex(), result);
3894           }
3895         } catch (IOException ioe) {
3896           // fail all the puts with the ioe in question.
3897           for (Action<R> a: mutations) {
3898             response.add(regionName, a.getOriginalIndex(), ioe);
3899           }
3900         }
3901       }
3902     }
3903     return response;
3904   }
3905 
3906   /**
3907    * Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}
3908    * method using the registered protocol handlers.
3909    * {@link CoprocessorProtocol} implementations must be registered per-region
3910    * via the
3911    * {@link org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)}
3912    * method before they are available.
3913    *
3914    * @param regionName name of the region against which the invocation is executed
3915    * @param call an {@code Exec} instance identifying the protocol, method name,
3916    *     and parameters for the method invocation
3917    * @return an {@code ExecResult} instance containing the region name of the
3918    *     invocation and the return value
3919    * @throws IOException if no registered protocol handler is found or an error
3920    *     occurs during the invocation
3921    * @see org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)
3922    */
3923   @Override
3924   public ExecResult execCoprocessor(byte[] regionName, Exec call)
3925       throws IOException {
3926     checkOpen();
3927     requestCount.incrementAndGet();
3928     try {
3929       HRegion region = getRegion(regionName);
3930       return region.exec(call);
3931     } catch (Throwable t) {
3932       throw convertThrowableToIOE(cleanup(t));
3933     }
3934   }
3935 
3936   public String toString() {
3937     return getServerName().toString();
3938   }
3939 
3940   /**
3941    * Interval at which threads should run
3942    *
3943    * @return the interval
3944    */
3945   public int getThreadWakeFrequency() {
3946     return threadWakeFrequency;
3947   }
3948 
3949   @Override
3950   public ZooKeeperWatcher getZooKeeper() {
3951     return zooKeeper;
3952   }
3953 
3954   @Override
3955   public ServerName getServerName() {
3956     // Our servername could change after we talk to the master.
3957     return this.serverNameFromMasterPOV == null?
3958       new ServerName(this.isa.getHostName(), this.isa.getPort(), this.startcode):
3959         this.serverNameFromMasterPOV;
3960   }
3961 
3962   @Override
3963   public CompactionRequestor getCompactionRequester() {
3964     return this.compactSplitThread;
3965   }
3966 
3967   public ZooKeeperWatcher getZooKeeperWatcher() {
3968     return this.zooKeeper;
3969   }
3970 
3971   public RegionServerCoprocessorHost getCoprocessorHost(){
3972     return this.rsHost;
3973   }
3974 
3975   @Override
3976   public boolean removeFromRegionsInTransition(final HRegionInfo hri) {
3977     return this.regionsInTransitionInRS.remove(hri.getEncodedNameAsBytes());
3978   }
3979 
3980   @Override
3981   public boolean containsKeyInRegionsInTransition(final HRegionInfo hri) {
3982     return this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes());
3983   }
3984 
3985   public ExecutorService getExecutorService() {
3986     return service;
3987   }
3988 
3989   //
3990   // Main program and support routines
3991   //
3992 
3993   /**
3994    * Load the replication service objects, if any
3995    */
3996   static private void createNewReplicationInstance(Configuration conf,
3997     HRegionServer server, FileSystem fs, Path logDir, Path oldLogDir) throws IOException{
3998 
3999     // If replication is not enabled, then return immediately.
4000     if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
4001       return;
4002     }
4003 
4004     // read in the name of the source replication class from the config file.
4005     String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
4006                                HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
4007 
4008     // read in the name of the sink replication class from the config file.
4009     String sinkClassname = conf.get(HConstants.REPLICATION_SINK_SERVICE_CLASSNAME,
4010                              HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
4011 
4012     // If both the sink and the source class names are the same, then instantiate
4013     // only one object.
4014     if (sourceClassname.equals(sinkClassname)) {
4015       server.replicationSourceHandler = (ReplicationSourceService)
4016                                          newReplicationInstance(sourceClassname,
4017                                          conf, server, fs, logDir, oldLogDir);
4018       server.replicationSinkHandler = (ReplicationSinkService)
4019                                          server.replicationSourceHandler;
4020     }
4021     else {
4022       server.replicationSourceHandler = (ReplicationSourceService)
4023                                          newReplicationInstance(sourceClassname,
4024                                          conf, server, fs, logDir, oldLogDir);
4025       server.replicationSinkHandler = (ReplicationSinkService)
4026                                          newReplicationInstance(sinkClassname,
4027                                          conf, server, fs, logDir, oldLogDir);
4028     }
4029   }
4030 
4031   static private ReplicationService newReplicationInstance(String classname,
4032     Configuration conf, HRegionServer server, FileSystem fs, Path logDir,
4033     Path oldLogDir) throws IOException{
4034 
4035     Class<?> clazz = null;
4036     try {
4037       ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
4038       clazz = Class.forName(classname, true, classLoader);
4039     } catch (java.lang.ClassNotFoundException nfe) {
4040       throw new IOException("Cound not find class for " + classname);
4041     }
4042 
4043     // create an instance of the replication object.
4044     ReplicationService service = (ReplicationService)
4045                               ReflectionUtils.newInstance(clazz, conf);
4046     service.initialize(server, fs, logDir, oldLogDir);
4047     return service;
4048   }
4049 
4050   /**
4051    * @param hrs
4052    * @return Thread the RegionServer is running in correctly named.
4053    * @throws IOException
4054    */
4055   public static Thread startRegionServer(final HRegionServer hrs)
4056       throws IOException {
4057     return startRegionServer(hrs, "regionserver" + hrs.isa.getPort());
4058   }
4059 
4060   /**
4061    * @param hrs
4062    * @param name
4063    * @return Thread the RegionServer is running in correctly named.
4064    * @throws IOException
4065    */
4066   public static Thread startRegionServer(final HRegionServer hrs,
4067       final String name) throws IOException {
4068     Thread t = new Thread(hrs);
4069     t.setName(name);
4070     t.start();
4071     // Install shutdown hook that will catch signals and run an orderly shutdown
4072     // of the hrs.
4073     ShutdownHook.install(hrs.getConfiguration(), FileSystem.get(hrs
4074         .getConfiguration()), hrs, t);
4075     return t;
4076   }
4077 
4078   /**
4079    * Utility for constructing an instance of the passed HRegionServer class.
4080    *
4081    * @param regionServerClass
4082    * @param conf2
4083    * @return HRegionServer instance.
4084    */
4085   public static HRegionServer constructRegionServer(
4086       Class<? extends HRegionServer> regionServerClass,
4087       final Configuration conf2) {
4088     try {
4089       Constructor<? extends HRegionServer> c = regionServerClass
4090           .getConstructor(Configuration.class);
4091       return c.newInstance(conf2);
4092     } catch (Exception e) {
4093       throw new RuntimeException("Failed construction of " + "Regionserver: "
4094           + regionServerClass.toString(), e);
4095     }
4096   }
4097 
4098   @Override
4099   @QosPriority(priority=HConstants.REPLICATION_QOS)
4100   public void replicateLogEntries(final HLog.Entry[] entries)
4101   throws IOException {
4102     checkOpen();
4103     if (this.replicationSinkHandler == null) return;
4104     this.replicationSinkHandler.replicateLogEntries(entries);
4105   }
4106 
4107   /**
4108    * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
4109    */
4110   public static void main(String[] args) throws Exception {
4111 	VersionInfo.logVersion();
4112     Configuration conf = HBaseConfiguration.create();
4113     @SuppressWarnings("unchecked")
4114     Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
4115         .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
4116 
4117     new HRegionServerCommandLine(regionServerClass).doMain(args);
4118   }
4119 
4120   @Override
4121   public List<BlockCacheColumnFamilySummary> getBlockCacheColumnFamilySummaries() throws IOException {
4122     BlockCache c = new CacheConfig(this.conf).getBlockCache();
4123     return c.getBlockCacheColumnFamilySummaries(this.conf);
4124   }
4125 
4126   @Override
4127   public byte[][] rollHLogWriter() throws IOException, FailedLogCloseException {
4128     HLog wal = this.getWAL();
4129     return wal.rollWriter(true);
4130   }
4131 
4132   /**
4133    * Gets the online regions of the specified table.
4134    * This method looks at the in-memory onlineRegions.  It does not go to <code>.META.</code>.
4135    * Only returns <em>online</em> regions.  If a region on this table has been
4136    * closed during a disable, etc., it will not be included in the returned list.
4137    * So, the returned list may not necessarily be ALL regions in this table, its
4138    * all the ONLINE regions in the table.
4139    * @param tableName
4140    * @return Online regions from <code>tableName</code>
4141    */
4142    public List<HRegion> getOnlineRegions(byte[] tableName) {
4143      List<HRegion> tableRegions = new ArrayList<HRegion>();
4144      synchronized (this.onlineRegions) {
4145        for (HRegion region: this.onlineRegions.values()) {
4146          HRegionInfo regionInfo = region.getRegionInfo();
4147          if(Bytes.equals(regionInfo.getTableName(), tableName)) {
4148            tableRegions.add(region);
4149          }
4150        }
4151      }
4152      return tableRegions;
4153    }
4154 
4155   // used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
4156   public String[] getCoprocessors() {
4157     TreeSet<String> coprocessors = new TreeSet<String>(
4158         this.hlog.getCoprocessorHost().getCoprocessors());
4159     Collection<HRegion> regions = getOnlineRegionsLocalContext();
4160     for (HRegion region: regions) {
4161       coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
4162     }
4163     return coprocessors.toArray(new String[0]);
4164   }
4165 
4166   /**
4167    * Register bean with platform management server
4168    */
4169   @SuppressWarnings("deprecation")
4170   void registerMBean() {
4171     MXBeanImpl mxBeanInfo = MXBeanImpl.init(this);
4172     mxBean = MBeanUtil.registerMBean("RegionServer", "RegionServer",
4173         mxBeanInfo);
4174     LOG.info("Registered RegionServer MXBean");
4175   }
4176 
4177   /**
4178    * Get the current compaction state of the region.
4179    *
4180    * @param regionName the name of the region to check compaction statte.
4181    * @return the compaction state name.
4182    * @throws IOException exception
4183    */
4184   public String getCompactionState(final byte[] regionName) throws IOException {
4185       checkOpen();
4186       requestCount.incrementAndGet();
4187       HRegion region = getRegion(regionName);
4188       HRegionInfo info = region.getRegionInfo();
4189       return CompactionRequest.getCompactionState(info.getRegionId()).name();
4190   }
4191 
4192   public long getResponseQueueSize(){
4193     if (server != null) {
4194       return server.getResponseQueueSize();
4195     }
4196     return 0;
4197   }
4198 
4199   private boolean isHealthCheckerConfigured() {
4200     String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
4201     return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
4202   }
4203 
4204   /**
4205    * @return the underlying {@link CompactSplitThread} for the servers
4206    */
4207   public CompactSplitThread getCompactSplitThread() {
4208     return this.compactSplitThread;
4209   }
4210 }