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