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.lang.Thread.UncaughtExceptionHandler;
24  import java.lang.annotation.Retention;
25  import java.lang.annotation.RetentionPolicy;
26  import java.lang.management.ManagementFactory;
27  import java.lang.management.MemoryUsage;
28  import java.lang.reflect.Constructor;
29  import java.lang.reflect.Method;
30  import java.net.BindException;
31  import java.net.InetSocketAddress;
32  import java.util.ArrayList;
33  import java.util.Collection;
34  import java.util.Collections;
35  import java.util.Comparator;
36  import java.util.HashMap;
37  import java.util.concurrent.ConcurrentHashMap;
38  import java.util.HashSet;
39  import java.util.Iterator;
40  import java.util.LinkedList;
41  import java.util.List;
42  import java.util.Map;
43  import java.util.Random;
44  import java.util.Set;
45  import java.util.SortedMap;
46  import java.util.TreeMap;
47  import java.util.concurrent.ConcurrentHashMap;
48  import java.util.concurrent.LinkedBlockingQueue;
49  import java.util.concurrent.TimeUnit;
50  import java.util.concurrent.atomic.AtomicBoolean;
51  import java.util.concurrent.atomic.AtomicInteger;
52  import java.util.concurrent.locks.ReentrantReadWriteLock;
53  
54  import com.google.common.collect.Lists;
55  import org.apache.commons.logging.Log;
56  import org.apache.commons.logging.LogFactory;
57  import org.apache.hadoop.conf.Configuration;
58  import org.apache.hadoop.fs.FileSystem;
59  import org.apache.hadoop.fs.Path;
60  import org.apache.hadoop.hbase.Chore;
61  import org.apache.hadoop.hbase.ClockOutOfSyncException;
62  import org.apache.hadoop.hbase.DoNotRetryIOException;
63  import org.apache.hadoop.hbase.HBaseConfiguration;
64  import org.apache.hadoop.hbase.HConstants;
65  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
66  import org.apache.hadoop.hbase.HMsg;
67  import org.apache.hadoop.hbase.HRegionInfo;
68  import org.apache.hadoop.hbase.HServerAddress;
69  import org.apache.hadoop.hbase.HServerInfo;
70  import org.apache.hadoop.hbase.HServerLoad;
71  import org.apache.hadoop.hbase.KeyValue;
72  import org.apache.hadoop.hbase.MasterAddressTracker;
73  import org.apache.hadoop.hbase.NotServingRegionException;
74  import org.apache.hadoop.hbase.RemoteExceptionHandler;
75  import org.apache.hadoop.hbase.Server;
76  import org.apache.hadoop.hbase.Stoppable;
77  import org.apache.hadoop.hbase.UnknownRowLockException;
78  import org.apache.hadoop.hbase.UnknownScannerException;
79  import org.apache.hadoop.hbase.YouAreDeadException;
80  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
81  import org.apache.hadoop.hbase.catalog.CatalogTracker;
82  import org.apache.hadoop.hbase.catalog.MetaEditor;
83  import org.apache.hadoop.hbase.catalog.RootLocationEditor;
84  import org.apache.hadoop.hbase.client.Action;
85  import org.apache.hadoop.hbase.client.Delete;
86  import org.apache.hadoop.hbase.client.Get;
87  import org.apache.hadoop.hbase.client.HConnection;
88  import org.apache.hadoop.hbase.client.HConnectionManager;
89  import org.apache.hadoop.hbase.client.Increment;
90  import org.apache.hadoop.hbase.client.MultiAction;
91  import org.apache.hadoop.hbase.client.MultiPut;
92  import org.apache.hadoop.hbase.client.MultiPutResponse;
93  import org.apache.hadoop.hbase.client.MultiResponse;
94  import org.apache.hadoop.hbase.client.Put;
95  import org.apache.hadoop.hbase.client.Result;
96  import org.apache.hadoop.hbase.client.Row;
97  import org.apache.hadoop.hbase.client.Scan;
98  import org.apache.hadoop.hbase.executor.ExecutorService;
99  import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
100 import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
101 import org.apache.hadoop.hbase.io.hfile.LruBlockCache.CacheStats;
102 import org.apache.hadoop.hbase.ipc.HBaseRPC;
103 import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
104 import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
105 import org.apache.hadoop.hbase.ipc.HBaseRpcMetrics;
106 import org.apache.hadoop.hbase.ipc.HBaseServer;
107 import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
108 import org.apache.hadoop.hbase.ipc.HRegionInterface;
109 import org.apache.hadoop.hbase.ipc.ServerNotRunningException;
110 import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
111 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
112 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
113 import org.apache.hadoop.hbase.regionserver.handler.CloseRootHandler;
114 import org.apache.hadoop.hbase.regionserver.handler.OpenMetaHandler;
115 import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
116 import org.apache.hadoop.hbase.regionserver.handler.OpenRootHandler;
117 import org.apache.hadoop.hbase.regionserver.metrics.RegionServerMetrics;
118 import org.apache.hadoop.hbase.regionserver.wal.HLog;
119 import org.apache.hadoop.hbase.regionserver.wal.WALObserver;
120 import org.apache.hadoop.hbase.replication.regionserver.Replication;
121 import org.apache.hadoop.hbase.security.User;
122 import org.apache.hadoop.hbase.util.Bytes;
123 import org.apache.hadoop.hbase.util.CompressionTest;
124 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
125 import org.apache.hadoop.hbase.util.FSUtils;
126 import org.apache.hadoop.hbase.util.InfoServer;
127 import org.apache.hadoop.hbase.util.Pair;
128 import org.apache.hadoop.hbase.util.Sleeper;
129 import org.apache.hadoop.hbase.util.Threads;
130 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
131 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
132 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
133 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
134 import org.apache.hadoop.io.MapWritable;
135 import org.apache.hadoop.io.Writable;
136 import org.apache.hadoop.ipc.RemoteException;
137 import org.apache.hadoop.net.DNS;
138 import org.apache.zookeeper.KeeperException;
139 
140 import com.google.common.base.Function;
141 
142 /**
143  * HRegionServer makes a set of HRegions available to clients. It checks in with
144  * the HMaster. There are many HRegionServers in a single HBase deployment.
145  */
146 public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
147     Runnable, RegionServerServices, Server {
148   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
149 
150   // Set when a report to the master comes back with a message asking us to
151   // shutdown. Also set by call to stop when debugging or running unit tests
152   // of HRegionServer in isolation.
153   protected volatile boolean stopped = false;
154 
155   // A state before we go into stopped state.  At this stage we're closing user
156   // space regions.
157   private boolean stopping = false;
158 
159   // Go down hard. Used if file system becomes unavailable and also in
160   // debugging and unit tests.
161   protected volatile boolean abortRequested;
162 
163   private volatile boolean killed = false;
164 
165   // If false, the file system has become unavailable
166   protected volatile boolean fsOk;
167 
168   protected HServerInfo serverInfo;
169   protected final Configuration conf;
170 
171   private final HConnection connection;
172   protected final AtomicBoolean haveRootRegion = new AtomicBoolean(false);
173   private FileSystem fs;
174   private Path rootDir;
175   private final Random rand = new Random();
176 
177   /**
178    * Map of regions currently being served by this region server. Key is the
179    * encoded region name.  All access should be synchronized.
180    */
181   protected final Map<String, HRegion> onlineRegions =
182     new ConcurrentHashMap<String, HRegion>();
183 
184   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
185   private final LinkedBlockingQueue<HMsg> outboundMsgs = new LinkedBlockingQueue<HMsg>();
186 
187   final int numRetries;
188   protected final int threadWakeFrequency;
189   private final int msgInterval;
190 
191   protected final int numRegionsToReport;
192 
193   private final long maxScannerResultSize;
194 
195   // Remote HMaster
196   private HMasterRegionInterface hbaseMaster;
197 
198   // Server to handle client requests. Default access so can be accessed by
199   // unit tests.
200   HBaseServer server;
201 
202   // Leases
203   private Leases leases;
204 
205   // Request counter
206   private volatile AtomicInteger requestCount = new AtomicInteger();
207 
208   // Info server. Default access so can be used by unit tests. REGIONSERVER
209   // is name of the webapp and the attribute name used stuffing this instance
210   // into web context.
211   InfoServer infoServer;
212 
213   /** region server process name */
214   public static final String REGIONSERVER = "regionserver";
215 
216   /*
217    * Space is reserved in HRS constructor and then released when aborting to
218    * recover from an OOME. See HBASE-706. TODO: Make this percentage of the heap
219    * or a minimum.
220    */
221   private final LinkedList<byte[]> reservedSpace = new LinkedList<byte[]>();
222 
223   private RegionServerMetrics metrics;
224 
225   // Compactions
226   CompactSplitThread compactSplitThread;
227 
228   // Cache flushing
229   MemStoreFlusher cacheFlusher;
230 
231   /*
232    * Check for major compactions.
233    */
234   Chore majorCompactionChecker;
235 
236   // HLog and HLog roller. log is protected rather than private to avoid
237   // eclipse warning when accessed by inner classes
238   protected volatile HLog hlog;
239   LogRoller hlogRoller;
240 
241   // flag set after we're done setting up server threads (used for testing)
242   protected volatile boolean isOnline;
243 
244   final Map<String, InternalScanner> scanners = new ConcurrentHashMap<String, InternalScanner>();
245 
246   // zookeeper connection and watcher
247   private ZooKeeperWatcher zooKeeper;
248 
249   // master address manager and watcher
250   private MasterAddressTracker masterAddressManager;
251 
252   // catalog tracker
253   private CatalogTracker catalogTracker;
254 
255   // Cluster Status Tracker
256   private ClusterStatusTracker clusterStatusTracker;
257 
258   // A sleeper that sleeps for msgInterval.
259   private final Sleeper sleeper;
260 
261   private final int rpcTimeout;
262 
263   // The main region server thread.
264   @SuppressWarnings("unused")
265   private Thread regionServerThread;
266 
267   // Instance of the hbase executor service.
268   private ExecutorService service;
269 
270   // Replication services. If no replication, this handler will be null.
271   private Replication replicationHandler;
272 
273   /**
274    * Starts a HRegionServer at the default location
275    *
276    * @param conf
277    * @throws IOException
278    * @throws InterruptedException
279    */
280   public HRegionServer(Configuration conf) throws IOException, InterruptedException {
281     this.fsOk = true;
282     this.conf = conf;
283     this.connection = HConnectionManager.getConnection(conf);
284     this.isOnline = false;
285 
286     // check to see if the codec list is available:
287     String [] codecs = conf.getStrings("hbase.regionserver.codecs",
288         (String[])null);
289     if (codecs != null) {
290       for (String codec : codecs) {
291         if (!CompressionTest.testCompression(codec)) {
292           throw new IOException("Compression codec " + codec +
293               " not supported, aborting RS construction");
294         }
295       }
296     }
297 
298     // Config'ed params
299     this.numRetries = conf.getInt("hbase.client.retries.number", 10);
300     this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY,
301         10 * 1000);
302     this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
303 
304     sleeper = new Sleeper(this.msgInterval, this);
305 
306     this.maxScannerResultSize = conf.getLong(
307         HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
308         HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
309 
310     this.numRegionsToReport = conf.getInt(
311         "hbase.regionserver.numregionstoreport", 10);
312 
313     this.rpcTimeout = conf.getInt(
314         HConstants.HBASE_RPC_TIMEOUT_KEY,
315         HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
316 
317     this.abortRequested = false;
318     this.stopped = false;
319 
320     // Server to handle client requests
321     String machineName = DNS.getDefaultHost(conf.get(
322         "hbase.regionserver.dns.interface", "default"), conf.get(
323         "hbase.regionserver.dns.nameserver", "default"));
324     String addressStr = machineName + ":" +
325       conf.get(HConstants.REGIONSERVER_PORT,
326         Integer.toString(HConstants.DEFAULT_REGIONSERVER_PORT));
327     HServerAddress address = new HServerAddress(addressStr);
328     this.server = HBaseRPC.getServer(this,
329         new Class<?>[]{HRegionInterface.class, HBaseRPCErrorHandler.class,
330         OnlineRegions.class},
331         address.getBindAddress(),
332       address.getPort(), conf.getInt("hbase.regionserver.handler.count", 10),
333         conf.getInt("hbase.regionserver.metahandler.count", 10),
334         false, conf, QOS_THRESHOLD);
335     this.server.setErrorHandler(this);
336     this.server.setQosFunction(new QosFunction());
337 
338     // HServerInfo can be amended by master.  See below in reportForDuty.
339     this.serverInfo = new HServerInfo(new HServerAddress(new InetSocketAddress(
340         address.getBindAddress(), this.server.getListenerAddress().getPort())),
341         System.currentTimeMillis(), this.conf.getInt(
342             "hbase.regionserver.info.port", 60030), machineName);
343     if (this.serverInfo.getServerAddress() == null) {
344       throw new NullPointerException("Server address cannot be null; "
345           + "hbase-958 debugging");
346     }
347 
348     // login the server principal (if using secure Hadoop)
349     User.login(conf, "hbase.regionserver.keytab.file",
350         "hbase.regionserver.kerberos.principal", serverInfo.getHostname());
351   }
352 
353   private static final int NORMAL_QOS = 0;
354   private static final int QOS_THRESHOLD = 10;  // the line between low and high qos
355   private static final int HIGH_QOS = 100;
356 
357   @Retention(RetentionPolicy.RUNTIME)
358   private @interface QosPriority {
359     int priority() default 0;
360   }
361 
362   class QosFunction implements Function<Writable,Integer> {
363     private final Map<String, Integer> annotatedQos;
364 
365     public QosFunction() {
366       Map<String, Integer> qosMap = new HashMap<String, Integer>();
367       for (Method m : HRegionServer.class.getMethods()) {
368         QosPriority p = m.getAnnotation(QosPriority.class);
369         if (p != null) {
370           qosMap.put(m.getName(), p.priority());
371         }
372       }
373       
374       annotatedQos = qosMap;
375     }
376 
377     public boolean isMetaRegion(byte[] regionName) {
378       HRegion region;
379       try {
380         region = getRegion(regionName);
381       } catch (NotServingRegionException ignored) {
382         return false;
383       }
384       return region.getRegionInfo().isMetaRegion();
385     }
386 
387     @Override
388     public Integer apply(Writable from) {
389       if (!(from instanceof HBaseRPC.Invocation)) return NORMAL_QOS;
390 
391       HBaseRPC.Invocation inv = (HBaseRPC.Invocation) from;
392       String methodName = inv.getMethodName();
393       
394       Integer priorityByAnnotation = annotatedQos.get(methodName);
395       if (priorityByAnnotation != null) {
396         return priorityByAnnotation;
397       }
398 
399       // scanner methods...
400       if (methodName.equals("next") || methodName.equals("close")) {
401         // translate!
402         Long scannerId;
403         try {
404           scannerId = (Long) inv.getParameters()[0];
405         } catch (ClassCastException ignored) {
406           // LOG.debug("Low priority: " + from);
407           return NORMAL_QOS; // doh.
408         }
409         String scannerIdString = Long.toString(scannerId);
410         InternalScanner scanner = scanners.get(scannerIdString);
411         if (scanner instanceof HRegion.RegionScanner) {
412           HRegion.RegionScanner rs = (HRegion.RegionScanner) scanner;
413           HRegionInfo regionName = rs.getRegionName();
414           if (regionName.isMetaRegion()) {
415             // LOG.debug("High priority scanner request: " + scannerId);
416             return HIGH_QOS;
417           }
418         }
419       } else if (inv.getParameterClasses().length == 0) {
420        // Just let it through.  This is getOnlineRegions, etc.
421       } else if (inv.getParameterClasses()[0] == byte[].class) {
422         // first arg is byte array, so assume this is a regionname:
423         if (isMetaRegion((byte[]) inv.getParameters()[0])) {
424           // LOG.debug("High priority with method: " + methodName +
425           // " and region: "
426           // + Bytes.toString((byte[]) inv.getParameters()[0]));
427           return HIGH_QOS;
428         }
429       } else if (inv.getParameterClasses()[0] == MultiAction.class) {
430         MultiAction ma = (MultiAction) inv.getParameters()[0];
431         Set<byte[]> regions = ma.getRegions();
432         // ok this sucks, but if any single of the actions touches a meta, the
433         // whole
434         // thing gets pingged high priority. This is a dangerous hack because
435         // people
436         // can get their multi action tagged high QOS by tossing a Get(.META.)
437         // AND this
438         // regionserver hosts META/-ROOT-
439         for (byte[] region : regions) {
440           if (isMetaRegion(region)) {
441             // LOG.debug("High priority multi with region: " +
442             // Bytes.toString(region));
443             return HIGH_QOS; // short circuit for the win.
444           }
445         }
446       }
447       // LOG.debug("Low priority: " + from.toString());
448       return NORMAL_QOS;
449     }
450   }
451 
452   /**
453    * Creates all of the state that needs to be reconstructed in case we are
454    * doing a restart. This is shared between the constructor and restart(). Both
455    * call it.
456    *
457    * @throws IOException
458    * @throws InterruptedException
459    */
460   private void initialize() throws IOException, InterruptedException {
461     try {
462       initializeZooKeeper();
463       initializeThreads();
464       int nbBlocks = conf.getInt("hbase.regionserver.nbreservationblocks", 4);
465       for (int i = 0; i < nbBlocks; i++) {
466         reservedSpace.add(new byte[HConstants.DEFAULT_SIZE_RESERVATION_BLOCK]);
467       }
468     } catch (Throwable t) {
469       // Call stop if error or process will stick around for ever since server
470       // puts up non-daemon threads.
471       LOG.error("Stopping HRS because failed initialize", t);
472       this.server.stop();
473     }
474   }
475 
476   /**
477    * Bring up connection to zk ensemble and then wait until a master for this
478    * cluster and then after that, wait until cluster 'up' flag has been set.
479    * This is the order in which master does things.
480    * Finally put up a catalog tracker.
481    * @throws IOException
482    * @throws InterruptedException
483    */
484   private void initializeZooKeeper() throws IOException, InterruptedException {
485     // Open connection to zookeeper and set primary watcher
486     zooKeeper = new ZooKeeperWatcher(conf, REGIONSERVER + ":" +
487       serverInfo.getServerAddress().getPort(), this);
488 
489     // Create the master address manager, register with zk, and start it.  Then
490     // block until a master is available.  No point in starting up if no master
491     // running.
492     this.masterAddressManager = new MasterAddressTracker(this.zooKeeper, this);
493     this.masterAddressManager.start();
494     blockAndCheckIfStopped(this.masterAddressManager);
495 
496     // Wait on cluster being up.  Master will set this flag up in zookeeper
497     // when ready.
498     this.clusterStatusTracker = new ClusterStatusTracker(this.zooKeeper, this);
499     this.clusterStatusTracker.start();
500     blockAndCheckIfStopped(this.clusterStatusTracker);
501 
502     // Create the catalog tracker and start it;
503     this.catalogTracker = new CatalogTracker(this.zooKeeper, this.connection,
504       this, this.conf.getInt("hbase.regionserver.catalog.timeout", Integer.MAX_VALUE));
505     catalogTracker.start();
506   }
507 
508   /**
509    * Utilty method to wait indefinitely on a znode availability while checking
510    * if the region server is shut down
511    * @param tracker znode tracker to use
512    * @throws IOException any IO exception, plus if the RS is stopped
513    * @throws InterruptedException
514    */
515   private void blockAndCheckIfStopped(ZooKeeperNodeTracker tracker)
516       throws IOException, InterruptedException {
517     while (tracker.blockUntilAvailable(this.msgInterval) == null) {
518       if (this.stopped) {
519         throw new IOException("Received the shutdown message while waiting.");
520       }
521     }
522   }
523 
524   /**
525    * @return False if cluster shutdown in progress
526    */
527   private boolean isClusterUp() {
528     return this.clusterStatusTracker.isClusterUp();
529   }
530 
531   private void initializeThreads() throws IOException {
532 
533     // Cache flushing thread.
534     this.cacheFlusher = new MemStoreFlusher(conf, this);
535 
536     // Compaction thread
537     this.compactSplitThread = new CompactSplitThread(this);
538 
539     // Background thread to check for major compactions; needed if region
540     // has not gotten updates in a while. Make it run at a lesser frequency.
541     int multiplier = this.conf.getInt(HConstants.THREAD_WAKE_FREQUENCY
542         + ".multiplier", 1000);
543     this.majorCompactionChecker = new MajorCompactionChecker(this,
544         this.threadWakeFrequency * multiplier, this);
545 
546     this.leases = new Leases((int) conf.getLong(
547         HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY,
548         HConstants.DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD),
549         this.threadWakeFrequency);
550   }
551 
552   /**
553    * The HRegionServer sticks in this loop until closed. It repeatedly checks in
554    * with the HMaster, sending heartbeats & reports, and receiving HRegion
555    * load/unload instructions.
556    */
557   public void run() {
558 
559     try {
560       // Initialize threads and wait for a master
561       initialize();
562     } catch (Exception e) {
563       abort("Fatal exception during initialization", e);
564     }
565 
566     this.regionServerThread = Thread.currentThread();
567     try {
568       while (!this.stopped) {
569         if (tryReportForDuty()) break;
570       }
571       long lastMsg = 0;
572       List<HMsg> outboundMessages = new ArrayList<HMsg>();
573       // The main run loop.
574       for (int tries = 0; !this.stopped && isHealthy();) {
575         if (!isClusterUp()) {
576           if (isOnlineRegionsEmpty()) {
577             stop("Exiting; cluster shutdown set and not carrying any regions");
578           } else if (!this.stopping) {
579             this.stopping = true;
580             closeUserRegions(this.abortRequested);
581           } else if (this.stopping && LOG.isDebugEnabled()) {
582             LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
583           }
584         }
585         long now = System.currentTimeMillis();
586         // Drop into the send loop if msgInterval has elapsed or if something
587         // to send. If we fail talking to the master, then we'll sleep below
588         // on poll of the outboundMsgs blockingqueue.
589         if ((now - lastMsg) >= msgInterval || !outboundMessages.isEmpty()) {
590           try {
591             doMetrics();
592             tryRegionServerReport(outboundMessages);
593             lastMsg = System.currentTimeMillis();
594             // Reset tries count if we had a successful transaction.
595             tries = 0;
596             if (this.stopped) continue;
597           } catch (Exception e) { // FindBugs REC_CATCH_EXCEPTION
598             // Two special exceptions could be printed out here,
599             // PleaseHoldException and YouAreDeadException
600             if (e instanceof IOException) {
601               e = RemoteExceptionHandler.checkIOException((IOException) e);
602             }
603             if (e instanceof YouAreDeadException) {
604               // This will be caught and handled as a fatal error below
605               throw e;
606             }
607             tries++;
608             if (tries > 0 && (tries % this.numRetries) == 0) {
609               // Check filesystem every so often.
610               checkFileSystem();
611             }
612             if (this.stopped) {
613               continue;
614             }
615             LOG.warn("Attempt=" + tries, e);
616             // No point retrying immediately; this is probably connection to
617             // master issue. Doing below will cause us to sleep.
618             lastMsg = System.currentTimeMillis();
619           }
620         }
621         now = System.currentTimeMillis();
622         HMsg msg = this.outboundMsgs.poll((msgInterval - (now - lastMsg)), TimeUnit.MILLISECONDS);
623         if (msg != null) outboundMessages.add(msg);
624       } // for
625     } catch (Throwable t) {
626       if (!checkOOME(t)) {
627         abort("Unhandled exception: " + t.getMessage(), t);
628       }
629     }
630     this.leases.closeAfterLeasesExpire();
631     this.server.stop();
632     if (this.infoServer != null) {
633       LOG.info("Stopping infoServer");
634       try {
635         this.infoServer.stop();
636       } catch (Exception e) {
637         e.printStackTrace();
638       }
639     }
640     // Send cache a shutdown.
641     LruBlockCache c = (LruBlockCache) StoreFile.getBlockCache(this.conf);
642     if (c != null) {
643       c.shutdown();
644     }
645 
646     // Send interrupts to wake up threads if sleeping so they notice shutdown.
647     // TODO: Should we check they are alive? If OOME could have exited already
648     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
649     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
650     if (this.hlogRoller != null) this.hlogRoller.interruptIfNecessary();
651     if (this.majorCompactionChecker != null) this.majorCompactionChecker.interrupt();
652 
653     if (this.killed) {
654       // Just skip out w/o closing regions.
655     } else if (abortRequested) {
656       if (this.fsOk) {
657         closeAllRegions(abortRequested); // Don't leave any open file handles
658         closeWAL(false);
659       }
660       LOG.info("aborting server at: " + this.serverInfo.getServerName());
661     } else {
662       closeAllRegions(abortRequested);
663       closeWAL(true);
664       closeAllScanners();
665       LOG.info("stopping server at: " + this.serverInfo.getServerName());
666     }
667     // Interrupt catalog tracker here in case any regions being opened out in
668     // handlers are stuck waiting on meta or root.
669     if (this.catalogTracker != null) this.catalogTracker.stop();
670     if (this.fsOk) waitOnAllRegionsToClose();
671 
672     // Make sure the proxy is down.
673     if (this.hbaseMaster != null) {
674       HBaseRPC.stopProxy(this.hbaseMaster);
675       this.hbaseMaster = null;
676     }
677     this.leases.close();
678     HConnectionManager.deleteConnection(conf, true);
679     this.zooKeeper.close();
680     if (!killed) {
681       join();
682     }
683     LOG.info(Thread.currentThread().getName() + " exiting");
684   }
685 
686   String getOnlineRegionsAsPrintableString() {
687     StringBuilder sb = new StringBuilder();
688     for (HRegion r: this.onlineRegions.values()) {
689       if (sb.length() > 0) sb.append(", ");
690       sb.append(r.getRegionInfo().getEncodedName());
691     }
692     return sb.toString();
693   }
694 
695   /**
696    * Wait on regions close.
697    */
698   private void waitOnAllRegionsToClose() {
699     // Wait till all regions are closed before going out.
700     int lastCount = -1;
701     while (!isOnlineRegionsEmpty()) {
702       int count = getNumberOfOnlineRegions();
703       // Only print a message if the count of regions has changed.
704       if (count != lastCount) {
705         lastCount = count;
706         LOG.info("Waiting on " + count + " regions to close");
707         // Only print out regions still closing if a small number else will
708         // swamp the log.
709         if (count < 10 && LOG.isDebugEnabled()) {
710           LOG.debug(this.onlineRegions);
711         }
712       }
713       Threads.sleep(1000);
714     }
715   }
716 
717   List<HMsg> tryRegionServerReport(final List<HMsg> outboundMessages)
718   throws IOException {
719     this.serverInfo.setLoad(buildServerLoad());
720     this.requestCount.set(0);
721     addOutboundMsgs(outboundMessages);
722     HMsg [] msgs = null;
723     while (!this.stopped) {
724       try {
725         msgs = this.hbaseMaster.regionServerReport(this.serverInfo,
726           outboundMessages.toArray(HMsg.EMPTY_HMSG_ARRAY),
727           getMostLoadedRegions());
728         break;
729       } catch (IOException ioe) {
730         if (ioe instanceof RemoteException) {
731           ioe = ((RemoteException)ioe).unwrapRemoteException();
732         }
733         if (ioe instanceof YouAreDeadException) {
734           // This will be caught and handled as a fatal error in run()
735           throw ioe;
736         }
737         // Couldn't connect to the master, get location from zk and reconnect
738         // Method blocks until new master is found or we are stopped
739         getMaster();
740       }
741     }
742     updateOutboundMsgs(outboundMessages);
743     outboundMessages.clear();
744 
745     for (int i = 0; !this.stopped && msgs != null && i < msgs.length; i++) {
746       LOG.info(msgs[i].toString());
747       // Intercept stop regionserver messages
748       if (msgs[i].getType().equals(HMsg.Type.STOP_REGIONSERVER)) {
749         stop("Received " + msgs[i]);
750         continue;
751       }
752       LOG.warn("NOT PROCESSING " + msgs[i] + " -- WHY IS MASTER SENDING IT TO US?");
753     }
754     return outboundMessages;
755   }
756 
757   private HServerLoad buildServerLoad() {
758     MemoryUsage memory = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
759     HServerLoad hsl = new HServerLoad(requestCount.get(),
760       (int)(memory.getUsed() / 1024 / 1024),
761       (int) (memory.getMax() / 1024 / 1024));
762     for (HRegion r : this.onlineRegions.values()) {
763       hsl.addRegionInfo(createRegionLoad(r));
764     }
765     return hsl;
766   }
767 
768   private void closeWAL(final boolean delete) {
769     try {
770       if (this.hlog != null) {
771         if (delete) {
772           hlog.closeAndDelete();
773         } else {
774           hlog.close();
775         }
776       }
777     } catch (Throwable e) {
778       LOG.error("Close and delete failed", RemoteExceptionHandler.checkThrowable(e));
779     }
780   }
781 
782   private void closeAllScanners() {
783     // Close any outstanding scanners. Means they'll get an UnknownScanner
784     // exception next time they come in.
785     for (Map.Entry<String, InternalScanner> e : this.scanners.entrySet()) {
786       try {
787         e.getValue().close();
788       } catch (IOException ioe) {
789         LOG.warn("Closing scanner " + e.getKey(), ioe);
790       }
791     }
792   }
793 
794   /*
795    * Add to the passed <code>msgs</code> messages to pass to the master.
796    *
797    * @param msgs Current outboundMsgs array; we'll add messages to this List.
798    */
799   private void addOutboundMsgs(final List<HMsg> msgs) {
800     if (msgs.isEmpty()) {
801       this.outboundMsgs.drainTo(msgs);
802       return;
803     }
804     OUTER: for (HMsg m : this.outboundMsgs) {
805       for (HMsg mm : msgs) {
806         // Be careful don't add duplicates.
807         if (mm.equals(m)) {
808           continue OUTER;
809         }
810       }
811       msgs.add(m);
812     }
813   }
814 
815   /*
816    * Remove from this.outboundMsgs those messsages we sent the master.
817    *
818    * @param msgs Messages we sent the master.
819    */
820   private void updateOutboundMsgs(final List<HMsg> msgs) {
821     if (msgs.isEmpty()) {
822       return;
823     }
824     for (HMsg m : this.outboundMsgs) {
825       for (HMsg mm : msgs) {
826         if (mm.equals(m)) {
827           this.outboundMsgs.remove(m);
828           break;
829         }
830       }
831     }
832   }
833 
834   /*
835    * Run init. Sets up hlog and starts up all server threads.
836    *
837    * @param c Extra configuration.
838    */
839   protected void handleReportForDutyResponse(final MapWritable c) throws IOException {
840     try {
841       for (Map.Entry<Writable, Writable> e : c.entrySet()) {
842 
843         String key = e.getKey().toString();
844         // Use the address the master passed us
845         if (key.equals("hbase.regionserver.address")) {
846           HServerAddress hsa = (HServerAddress) e.getValue();
847           LOG.info("Master passed us address to use. Was="
848             + this.serverInfo.getServerAddress() + ", Now=" + hsa.toString());
849           this.serverInfo.setServerAddress(hsa);
850           continue;
851         }
852         String value = e.getValue().toString();
853         if (LOG.isDebugEnabled()) {
854           LOG.debug("Config from master: " + key + "=" + value);
855         }
856         this.conf.set(key, value);
857       }
858       // hack! Maps DFSClient => RegionServer for logs.  HDFS made this
859       // config param for task trackers, but we can piggyback off of it.
860       if (this.conf.get("mapred.task.id") == null) {
861         this.conf.set("mapred.task.id",
862             "hb_rs_" + this.serverInfo.getServerName() + "_" +
863             System.currentTimeMillis());
864       }
865 
866       // Master sent us hbase.rootdir to use. Should be fully qualified
867       // path with file system specification included. Set 'fs.defaultFS'
868       // to match the filesystem on hbase.rootdir else underlying hadoop hdfs
869       // accessors will be going against wrong filesystem (unless all is set
870       // to defaults).
871       this.conf.set("fs.defaultFS", this.conf.get("hbase.rootdir"));
872       // Get fs instance used by this RS
873       this.fs = FileSystem.get(this.conf);
874       this.rootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
875       this.hlog = setupWALAndReplication();
876       // Init in here rather than in constructor after thread name has been set
877       this.metrics = new RegionServerMetrics();
878       startServiceThreads();
879       LOG.info("Serving as " + this.serverInfo.getServerName() +
880         ", RPC listening on " + this.server.getListenerAddress() +
881         ", sessionid=0x" +
882         Long.toHexString(this.zooKeeper.getZooKeeper().getSessionId()));
883       isOnline = true;
884     } catch (Throwable e) {
885       this.isOnline = false;
886       stop("Failed initialization");
887       throw convertThrowableToIOE(cleanup(e, "Failed init"),
888           "Region server startup failed");
889     }
890   }
891 
892   /*
893    * @param r Region to get RegionLoad for.
894    *
895    * @return RegionLoad instance.
896    *
897    * @throws IOException
898    */
899   private HServerLoad.RegionLoad createRegionLoad(final HRegion r) {
900     byte[] name = r.getRegionName();
901     int stores = 0;
902     int storefiles = 0;
903     int storefileSizeMB = 0;
904     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
905     int storefileIndexSizeMB = 0;
906     synchronized (r.stores) {
907       stores += r.stores.size();
908       for (Store store : r.stores.values()) {
909         storefiles += store.getStorefilesCount();
910         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
911         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
912       }
913     }
914     return new HServerLoad.RegionLoad(name, stores, storefiles,
915         storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB);
916   }
917 
918   /**
919    * @param encodedRegionName
920    * @return An instance of RegionLoad.
921    * @throws IOException
922    */
923   public HServerLoad.RegionLoad createRegionLoad(final String encodedRegionName) {
924     HRegion r = null;
925     r = this.onlineRegions.get(encodedRegionName);
926     return r != null ? createRegionLoad(r) : null;
927   }
928 
929   /*
930    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
931    * IOE if it isn't already.
932    *
933    * @param t Throwable
934    *
935    * @return Throwable converted to an IOE; methods can only let out IOEs.
936    */
937   private Throwable cleanup(final Throwable t) {
938     return cleanup(t, null);
939   }
940 
941   /*
942    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
943    * IOE if it isn't already.
944    *
945    * @param t Throwable
946    *
947    * @param msg Message to log in error. Can be null.
948    *
949    * @return Throwable converted to an IOE; methods can only let out IOEs.
950    */
951   private Throwable cleanup(final Throwable t, final String msg) {
952     // Don't log as error if NSRE; NSRE is 'normal' operation.
953     if (t instanceof NotServingRegionException) {
954       LOG.debug("NotServingRegionException; " +  t.getMessage());
955       return t;
956     }
957     if (msg == null) {
958       LOG.error("", RemoteExceptionHandler.checkThrowable(t));
959     } else {
960       LOG.error(msg, RemoteExceptionHandler.checkThrowable(t));
961     }
962     if (!checkOOME(t)) {
963       checkFileSystem();
964     }
965     return t;
966   }
967 
968   /*
969    * @param t
970    *
971    * @return Make <code>t</code> an IOE if it isn't already.
972    */
973   private IOException convertThrowableToIOE(final Throwable t) {
974     return convertThrowableToIOE(t, null);
975   }
976 
977   /*
978    * @param t
979    *
980    * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
981    *
982    * @return Make <code>t</code> an IOE if it isn't already.
983    */
984   private IOException convertThrowableToIOE(final Throwable t, final String msg) {
985     return (t instanceof IOException ? (IOException) t : msg == null
986         || msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
987   }
988 
989   /*
990    * Check if an OOME and if so, call abort.
991    *
992    * @param e
993    *
994    * @return True if we OOME'd and are aborting.
995    */
996   public boolean checkOOME(final Throwable e) {
997     boolean stop = false;
998     if (e instanceof OutOfMemoryError
999         || (e.getCause() != null && e.getCause() instanceof OutOfMemoryError)
1000         || (e.getMessage() != null && e.getMessage().contains(
1001             "java.lang.OutOfMemoryError"))) {
1002       abort("OutOfMemoryError, aborting", e);
1003       stop = true;
1004     }
1005     return stop;
1006   }
1007 
1008   /**
1009    * Checks to see if the file system is still accessible. If not, sets
1010    * abortRequested and stopRequested
1011    *
1012    * @return false if file system is not available
1013    */
1014   protected boolean checkFileSystem() {
1015     if (this.fsOk && this.fs != null) {
1016       try {
1017         FSUtils.checkFileSystemAvailable(this.fs);
1018       } catch (IOException e) {
1019         abort("File System not available", e);
1020         this.fsOk = false;
1021       }
1022     }
1023     return this.fsOk;
1024   }
1025 
1026   /*
1027    * Inner class that runs on a long period checking if regions need major
1028    * compaction.
1029    */
1030   private static class MajorCompactionChecker extends Chore {
1031     private final HRegionServer instance;
1032 
1033     MajorCompactionChecker(final HRegionServer h, final int sleepTime,
1034         final Stoppable stopper) {
1035       super("MajorCompactionChecker", sleepTime, h);
1036       this.instance = h;
1037       LOG.info("Runs every " + sleepTime + "ms");
1038     }
1039 
1040     @Override
1041     protected void chore() {
1042       for (HRegion r : this.instance.onlineRegions.values()) {
1043         try {
1044           if (r != null && r.isMajorCompaction()) {
1045             // Queue a compaction. Will recognize if major is needed.
1046             this.instance.compactSplitThread.requestCompaction(r, getName()
1047               + " requests major compaction");
1048           }
1049         } catch (IOException e) {
1050           LOG.warn("Failed major compaction check on " + r, e);
1051         }
1052       }
1053     }
1054   }
1055 
1056   /**
1057    * Report the status of the server. A server is online once all the startup is
1058    * completed (setting up filesystem, starting service threads, etc.). This
1059    * method is designed mostly to be useful in tests.
1060    *
1061    * @return true if online, false if not.
1062    */
1063   public boolean isOnline() {
1064     return isOnline;
1065   }
1066 
1067   /**
1068    * Setup WAL log and replication if enabled.
1069    * Replication setup is done in here because it wants to be hooked up to WAL.
1070    * @return A WAL instance.
1071    * @throws IOException
1072    */
1073   private HLog setupWALAndReplication() throws IOException {
1074     final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1075     Path logdir = new Path(rootDir, HLog.getHLogDirectoryName(this.serverInfo));
1076     if (LOG.isDebugEnabled()) {
1077       LOG.debug("logdir=" + logdir);
1078     }
1079     if (this.fs.exists(logdir)) {
1080       throw new RegionServerRunningException("Region server already "
1081           + "running at " + this.serverInfo.getServerName()
1082           + " because logdir " + logdir.toString() + " exists");
1083     }
1084 
1085     // Instantiate replication manager if replication enabled.  Pass it the
1086     // log directories.
1087     try {
1088       this.replicationHandler = Replication.isReplication(this.conf)?
1089         new Replication(this, this.fs, logdir, oldLogDir): null;
1090     } catch (KeeperException e) {
1091       throw new IOException("Failed replication handler create", e);
1092     }
1093     return instantiateHLog(logdir, oldLogDir);
1094   }
1095 
1096   /**
1097    * Called by {@link #setupWALAndReplication()} creating WAL instance.
1098    * @param logdir
1099    * @param oldLogDir
1100    * @return WAL instance.
1101    * @throws IOException
1102    */
1103   protected HLog instantiateHLog(Path logdir, Path oldLogDir) throws IOException {
1104     return new HLog(this.fs, logdir, oldLogDir, this.conf,
1105       getWALActionListeners(), this.serverInfo.getServerAddress().toString());
1106   }
1107 
1108   /**
1109    * Called by {@link #instantiateHLog(Path, Path)} setting up WAL instance.
1110    * Add any {@link WALObserver}s you want inserted before WAL startup.
1111    * @return List of WALActionsListener that will be passed in to
1112    * {@link HLog} on construction.
1113    */
1114   protected List<WALObserver> getWALActionListeners() {
1115     List<WALObserver> listeners = new ArrayList<WALObserver>();
1116     // Log roller.
1117     this.hlogRoller = new LogRoller(this, this);
1118     listeners.add(this.hlogRoller);
1119     if (this.replicationHandler != null) {
1120       // Replication handler is an implementation of WALActionsListener.
1121       listeners.add(this.replicationHandler);
1122     }
1123     return listeners;
1124   }
1125 
1126   protected LogRoller getLogRoller() {
1127     return hlogRoller;
1128   }
1129 
1130   /*
1131    * @param interval Interval since last time metrics were called.
1132    */
1133   protected void doMetrics() {
1134     try {
1135       metrics();
1136     } catch (Throwable e) {
1137       LOG.warn("Failed metrics", e);
1138     }
1139   }
1140 
1141   protected void metrics() {
1142     this.metrics.regions.set(this.onlineRegions.size());
1143     this.metrics.incrementRequests(this.requestCount.get());
1144     // Is this too expensive every three seconds getting a lock on onlineRegions
1145     // and then per store carried? Can I make metrics be sloppier and avoid
1146     // the synchronizations?
1147     int stores = 0;
1148     int storefiles = 0;
1149     long memstoreSize = 0;
1150     long storefileIndexSize = 0;
1151     for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
1152         HRegion r = e.getValue();
1153         memstoreSize += r.memstoreSize.get();
1154         synchronized (r.stores) {
1155           stores += r.stores.size();
1156           for (Map.Entry<byte[], Store> ee : r.stores.entrySet()) {
1157             Store store = ee.getValue();
1158             storefiles += store.getStorefilesCount();
1159             storefileIndexSize += store.getStorefilesIndexSize();
1160           }
1161         }
1162       }
1163     this.metrics.stores.set(stores);
1164     this.metrics.storefiles.set(storefiles);
1165     this.metrics.memstoreSizeMB.set((int) (memstoreSize / (1024 * 1024)));
1166     this.metrics.storefileIndexSizeMB
1167         .set((int) (storefileIndexSize / (1024 * 1024)));
1168     this.metrics.compactionQueueSize.set(compactSplitThread
1169         .getCompactionQueueSize());
1170     this.metrics.flushQueueSize.set(cacheFlusher
1171         .getFlushQueueSize());
1172 
1173     LruBlockCache lruBlockCache = (LruBlockCache) StoreFile.getBlockCache(conf);
1174     if (lruBlockCache != null) {
1175       this.metrics.blockCacheCount.set(lruBlockCache.size());
1176       this.metrics.blockCacheFree.set(lruBlockCache.getFreeSize());
1177       this.metrics.blockCacheSize.set(lruBlockCache.getCurrentSize());
1178       CacheStats cacheStats = lruBlockCache.getStats();
1179       this.metrics.blockCacheHitCount.set(cacheStats.getHitCount());
1180       this.metrics.blockCacheMissCount.set(cacheStats.getMissCount());
1181       this.metrics.blockCacheEvictedCount.set(lruBlockCache.getEvictedCount());
1182       double ratio = lruBlockCache.getStats().getHitRatio();
1183       int percent = (int) (ratio * 100);
1184       this.metrics.blockCacheHitRatio.set(percent);
1185       ratio = lruBlockCache.getStats().getHitCachingRatio();
1186       percent = (int) (ratio * 100);
1187       this.metrics.blockCacheHitCachingRatio.set(percent);
1188     }
1189   }
1190 
1191   /**
1192    * @return Region server metrics instance.
1193    */
1194   public RegionServerMetrics getMetrics() {
1195     return this.metrics;
1196   }
1197 
1198   /*
1199    * Start maintanence Threads, Server, Worker and lease checker threads.
1200    * Install an UncaughtExceptionHandler that calls abort of RegionServer if we
1201    * get an unhandled exception. We cannot set the handler on all threads.
1202    * Server's internal Listener thread is off limits. For Server, if an OOME, it
1203    * waits a while then retries. Meantime, a flush or a compaction that tries to
1204    * run should trigger same critical condition and the shutdown will run. On
1205    * its way out, this server will shut down Server. Leases are sort of
1206    * inbetween. It has an internal thread that while it inherits from Chore, it
1207    * keeps its own internal stop mechanism so needs to be stopped by this
1208    * hosting server. Worker logs the exception and exits.
1209    */
1210   private void startServiceThreads() throws IOException {
1211     String n = Thread.currentThread().getName();
1212     UncaughtExceptionHandler handler = new UncaughtExceptionHandler() {
1213       public void uncaughtException(Thread t, Throwable e) {
1214         abort("Uncaught exception in service thread " + t.getName(), e);
1215       }
1216     };
1217 
1218     // Start executor services
1219     this.service = new ExecutorService(getServerName());
1220     this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
1221       conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1222     this.service.startExecutorService(ExecutorType.RS_OPEN_ROOT,
1223       conf.getInt("hbase.regionserver.executor.openroot.threads", 1));
1224     this.service.startExecutorService(ExecutorType.RS_OPEN_META,
1225       conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1226     this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1227       conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1228     this.service.startExecutorService(ExecutorType.RS_CLOSE_ROOT,
1229       conf.getInt("hbase.regionserver.executor.closeroot.threads", 1));
1230     this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
1231       conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1232 
1233     Threads.setDaemonThreadRunning(this.hlogRoller, n + ".logRoller", handler);
1234     Threads.setDaemonThreadRunning(this.cacheFlusher, n + ".cacheFlusher",
1235         handler);
1236     Threads.setDaemonThreadRunning(this.compactSplitThread, n + ".compactor",
1237         handler);
1238     Threads.setDaemonThreadRunning(this.majorCompactionChecker, n
1239         + ".majorCompactionChecker", handler);
1240 
1241     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
1242     // an unhandled exception, it will just exit.
1243     this.leases.setName(n + ".leaseChecker");
1244     this.leases.start();
1245     // Put up info server.
1246     int port = this.conf.getInt("hbase.regionserver.info.port", 60030);
1247     // -1 is for disabling info server
1248     if (port >= 0) {
1249       String addr = this.conf.get("hbase.regionserver.info.bindAddress",
1250           "0.0.0.0");
1251       // check if auto port bind enabled
1252       boolean auto = this.conf.getBoolean("hbase.regionserver.info.port.auto",
1253           false);
1254       while (true) {
1255         try {
1256           this.infoServer = new InfoServer("regionserver", addr, port, false);
1257           this.infoServer.setAttribute("regionserver", this);
1258           this.infoServer.start();
1259           break;
1260         } catch (BindException e) {
1261           if (!auto) {
1262             // auto bind disabled throw BindException
1263             throw e;
1264           }
1265           // auto bind enabled, try to use another port
1266           LOG.info("Failed binding http info server to port: " + port);
1267           port++;
1268           // update HRS server info port.
1269           this.serverInfo = new HServerInfo(this.serverInfo.getServerAddress(),
1270             this.serverInfo.getStartCode(), port,
1271             this.serverInfo.getHostname());
1272         }
1273       }
1274     }
1275 
1276     if (this.replicationHandler != null) {
1277       this.replicationHandler.startReplicationServices();
1278     }
1279 
1280     // Start Server.  This service is like leases in that it internally runs
1281     // a thread.
1282     this.server.start();
1283   }
1284 
1285   /*
1286    * Verify that server is healthy
1287    */
1288   private boolean isHealthy() {
1289     if (!fsOk) {
1290       // File system problem
1291       return false;
1292     }
1293     // Verify that all threads are alive
1294     if (!(leases.isAlive() && compactSplitThread.isAlive()
1295         && cacheFlusher.isAlive() && hlogRoller.isAlive()
1296         && this.majorCompactionChecker.isAlive())) {
1297       stop("One or more threads are no longer alive -- stop");
1298       return false;
1299     }
1300     return true;
1301   }
1302 
1303   @Override
1304   public HLog getWAL() {
1305     return this.hlog;
1306   }
1307 
1308   @Override
1309   public CatalogTracker getCatalogTracker() {
1310     return this.catalogTracker;
1311   }
1312 
1313   @Override
1314   public void stop(final String msg) {
1315     this.stopped = true;
1316     LOG.info("STOPPED: " + msg);
1317     synchronized (this) {
1318       // Wakes run() if it is sleeping
1319       notifyAll(); // FindBugs NN_NAKED_NOTIFY
1320     }
1321   }
1322 
1323   @Override
1324   public void postOpenDeployTasks(final HRegion r, final CatalogTracker ct,
1325       final boolean daughter)
1326   throws KeeperException, IOException {
1327     // Do checks to see if we need to compact (references or too many files)
1328     if (r.hasReferences() || r.hasTooManyStoreFiles()) {
1329       getCompactionRequester().requestCompaction(r,
1330         r.hasReferences()? "Region has references on open" :
1331           "Region has too many store files");
1332     }
1333 
1334     // Add to online regions if all above was successful.
1335     addToOnlineRegions(r);
1336 
1337     // Update ZK, ROOT or META
1338     if (r.getRegionInfo().isRootRegion()) {
1339       RootLocationEditor.setRootLocation(getZooKeeper(),
1340         getServerInfo().getServerAddress());
1341     } else if (r.getRegionInfo().isMetaRegion()) {
1342       MetaEditor.updateMetaLocation(ct, r.getRegionInfo(), getServerInfo());
1343     } else {
1344       if (daughter) {
1345         // If daughter of a split, update whole row, not just location.
1346         MetaEditor.addDaughter(ct, r.getRegionInfo(), getServerInfo());
1347       } else {
1348         MetaEditor.updateRegionLocation(ct, r.getRegionInfo(), getServerInfo());
1349       }
1350     }
1351   }
1352 
1353   /**
1354    * Return a reference to the metrics instance used for counting RPC calls.
1355    * @return Metrics instance.
1356    */
1357   public HBaseRpcMetrics getRpcMetrics() {
1358     return server.getRpcMetrics();
1359   }
1360 
1361   /**
1362    * Cause the server to exit without closing the regions it is serving, the log
1363    * it is using and without notifying the master. Used unit testing and on
1364    * catastrophic events such as HDFS is yanked out from under hbase or we OOME.
1365    *
1366    * @param reason
1367    *          the reason we are aborting
1368    * @param cause
1369    *          the exception that caused the abort, or null
1370    */
1371   public void abort(String reason, Throwable cause) {
1372     if (cause != null) {
1373       LOG.fatal("ABORTING region server " + this + ": " + reason, cause);
1374     } else {
1375       LOG.fatal("ABORTING region server " + this + ": " + reason);
1376     }
1377     this.abortRequested = true;
1378     this.reservedSpace.clear();
1379     if (this.metrics != null) {
1380       LOG.info("Dump of metrics: " + this.metrics);
1381     }
1382     stop(reason);
1383   }
1384 
1385   /**
1386    * @see HRegionServer#abort(String, Throwable)
1387    */
1388   public void abort(String reason) {
1389     abort(reason, null);
1390   }
1391 
1392   /*
1393    * Simulate a kill -9 of this server. Exits w/o closing regions or cleaninup
1394    * logs but it does close socket in case want to bring up server on old
1395    * hostname+port immediately.
1396    */
1397   protected void kill() {
1398     this.killed = true;
1399     abort("Simulated kill");
1400   }
1401 
1402   /**
1403    * Wait on all threads to finish. Presumption is that all closes and stops
1404    * have already been called.
1405    */
1406   protected void join() {
1407     Threads.shutdown(this.majorCompactionChecker);
1408     Threads.shutdown(this.cacheFlusher);
1409     Threads.shutdown(this.compactSplitThread);
1410     Threads.shutdown(this.hlogRoller);
1411     this.service.shutdown();
1412     if (this.replicationHandler != null) {
1413       this.replicationHandler.join();
1414     }
1415   }
1416 
1417   /**
1418    * Get the current master from ZooKeeper and open the RPC connection to it.
1419    *
1420    * Method will block until a master is available. You can break from this
1421    * block by requesting the server stop.
1422    *
1423    * @return master address, or null if server has been stopped
1424    */
1425   private HServerAddress getMaster() {
1426     HServerAddress masterAddress = null;
1427     HMasterRegionInterface master = null;
1428 
1429     while (!stopped && master == null) {
1430 
1431       masterAddress = getMasterAddress();
1432       LOG.info("Attempting connect to Master server at " + masterAddress);
1433       try {
1434         // Do initial RPC setup. The final argument indicates that the RPC
1435         // should retry indefinitely.
1436         master = (HMasterRegionInterface) HBaseRPC.waitForProxy(
1437             HMasterRegionInterface.class, HBaseRPCProtocolVersion.versionID,
1438             masterAddress.getInetSocketAddress(), this.conf, -1,
1439             this.rpcTimeout, this.rpcTimeout);
1440       } catch (IOException e) {
1441         e = e instanceof RemoteException ?
1442             ((RemoteException)e).unwrapRemoteException() : e;
1443         if (e instanceof ServerNotRunningException) {
1444           LOG.info("Master isn't available yet, retrying");
1445         } else {
1446           LOG.warn("Unable to connect to master. Retrying. Error was:", e);
1447         }
1448         sleeper.sleep();
1449       }
1450     }
1451     LOG.info("Connected to master at " + masterAddress);
1452     this.hbaseMaster = master;
1453     return masterAddress;
1454   }
1455 
1456   private HServerAddress getMasterAddress() {
1457     HServerAddress masterAddress = null;
1458       while ((masterAddress = masterAddressManager.getMasterAddress()) == null) {
1459         if (stopped) {
1460           return null;
1461         }
1462         LOG.debug("No master found, will retry");
1463         sleeper.sleep();
1464       }
1465       return masterAddress;
1466    }
1467 
1468   /**
1469    * @return True if successfully invoked {@link #reportForDuty()}
1470    * @throws IOException
1471    */
1472   private boolean tryReportForDuty() throws IOException {
1473     MapWritable w = reportForDuty();
1474     if (w != null) {
1475       handleReportForDutyResponse(w);
1476       return true;
1477     }
1478     sleeper.sleep();
1479     LOG.warn("No response on reportForDuty. Sleeping and then retrying.");
1480     return false;
1481   }
1482 
1483   /*
1484    * Let the master know we're here Run initialization using parameters passed
1485    * us by the master.
1486    */
1487   private MapWritable reportForDuty() throws IOException {
1488     HServerAddress masterAddress = null;
1489     while (!stopped && (masterAddress = getMaster()) == null) {
1490       sleeper.sleep();
1491       LOG.warn("Unable to get master for initialization");
1492     }
1493 
1494     MapWritable result = null;
1495     long lastMsg = 0;
1496     while (!stopped) {
1497       try {
1498         this.requestCount.set(0);
1499         lastMsg = System.currentTimeMillis();
1500         ZKUtil.setAddressAndWatch(zooKeeper,
1501           ZKUtil.joinZNode(zooKeeper.rsZNode, ZKUtil.getNodeName(serverInfo)),
1502           this.serverInfo.getServerAddress());
1503         this.serverInfo.setLoad(buildServerLoad());
1504         LOG.info("Telling master at " + masterAddress + " that we are up");
1505         result = this.hbaseMaster.regionServerStartup(this.serverInfo,
1506             EnvironmentEdgeManager.currentTimeMillis());
1507         break;
1508       } catch (RemoteException e) {
1509         IOException ioe = e.unwrapRemoteException();
1510         if (ioe instanceof ClockOutOfSyncException) {
1511           LOG.fatal("Master rejected startup because clock is out of sync",
1512               ioe);
1513           // Re-throw IOE will cause RS to abort
1514           throw ioe;
1515         } else {
1516           LOG.warn("remote error telling master we are up", e);
1517         }
1518       } catch (IOException e) {
1519         LOG.warn("error telling master we are up", e);
1520       } catch (KeeperException e) {
1521         LOG.warn("error putting up ephemeral node in zookeeper", e);
1522       }
1523       sleeper.sleep(lastMsg);
1524     }
1525     return result;
1526   }
1527 
1528   /**
1529    * Add to the outbound message buffer
1530    *
1531    * When a region splits, we need to tell the master that there are two new
1532    * regions that need to be assigned.
1533    *
1534    * We do not need to inform the master about the old region, because we've
1535    * updated the meta or root regions, and the master will pick that up on its
1536    * next rescan of the root or meta tables.
1537    */
1538   void reportSplit(HRegionInfo oldRegion, HRegionInfo newRegionA,
1539       HRegionInfo newRegionB) {
1540     this.outboundMsgs.add(new HMsg(
1541         HMsg.Type.REGION_SPLIT, oldRegion, newRegionA,
1542         newRegionB, Bytes.toBytes("Daughters; "
1543             + newRegionA.getRegionNameAsString() + ", "
1544             + newRegionB.getRegionNameAsString())));
1545   }
1546 
1547   /**
1548    * Closes all regions.  Called on our way out.
1549    * Assumes that its not possible for new regions to be added to onlineRegions
1550    * while this method runs.
1551    */
1552   protected void closeAllRegions(final boolean abort) {
1553     closeUserRegions(abort);
1554     // Only root and meta should remain.  Are we carrying root or meta?
1555     HRegion meta = null;
1556     HRegion root = null;
1557     this.lock.writeLock().lock();
1558     try {
1559       for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
1560         HRegionInfo hri = e.getValue().getRegionInfo();
1561         if (hri.isRootRegion()) {
1562           root = e.getValue();
1563         } else if (hri.isMetaRegion()) {
1564           meta = e.getValue();
1565         }
1566         if (meta != null && root != null) break;
1567       }
1568     } finally {
1569       this.lock.writeLock().unlock();
1570     }
1571     if (meta != null) closeRegion(meta.getRegionInfo(), abort, false);
1572     if (root != null) closeRegion(root.getRegionInfo(), abort, false);
1573   }
1574 
1575   /**
1576    * Schedule closes on all user regions.
1577    * @param abort Whether we're running an abort.
1578    */
1579   void closeUserRegions(final boolean abort) {
1580     this.lock.writeLock().lock();
1581     try {
1582       for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
1583         HRegion r = e.getValue();
1584         if (!r.getRegionInfo().isMetaRegion()) {
1585           // Don't update zk with this close transition; pass false.
1586           closeRegion(r.getRegionInfo(), abort, false);
1587         }
1588       }
1589     } finally {
1590       this.lock.writeLock().unlock();
1591     }
1592   }
1593 
1594   @Override
1595   @QosPriority(priority=HIGH_QOS)
1596   public HRegionInfo getRegionInfo(final byte[] regionName)
1597   throws NotServingRegionException {
1598     requestCount.incrementAndGet();
1599     return getRegion(regionName).getRegionInfo();
1600   }
1601 
1602   public Result getClosestRowBefore(final byte[] regionName, final byte[] row,
1603       final byte[] family) throws IOException {
1604     checkOpen();
1605     requestCount.incrementAndGet();
1606     try {
1607       // locate the region we're operating on
1608       HRegion region = getRegion(regionName);
1609       // ask the region for all the data
1610 
1611       Result r = region.getClosestRowBefore(row, family);
1612       return r;
1613     } catch (Throwable t) {
1614       throw convertThrowableToIOE(cleanup(t));
1615     }
1616   }
1617 
1618   /** {@inheritDoc} */
1619   public Result get(byte[] regionName, Get get) throws IOException {
1620     checkOpen();
1621     requestCount.incrementAndGet();
1622     try {
1623       HRegion region = getRegion(regionName);
1624       return region.get(get, getLockFromId(get.getLockId()));
1625     } catch (Throwable t) {
1626       throw convertThrowableToIOE(cleanup(t));
1627     }
1628   }
1629 
1630   public boolean exists(byte[] regionName, Get get) throws IOException {
1631     checkOpen();
1632     requestCount.incrementAndGet();
1633     try {
1634       HRegion region = getRegion(regionName);
1635       Result r = region.get(get, getLockFromId(get.getLockId()));
1636       return r != null && !r.isEmpty();
1637     } catch (Throwable t) {
1638       throw convertThrowableToIOE(cleanup(t));
1639     }
1640   }
1641 
1642   public void put(final byte[] regionName, final Put put) throws IOException {
1643     if (put.getRow() == null) {
1644       throw new IllegalArgumentException("update has null row");
1645     }
1646 
1647     checkOpen();
1648     this.requestCount.incrementAndGet();
1649     HRegion region = getRegion(regionName);
1650     try {
1651       if (!region.getRegionInfo().isMetaTable()) {
1652         this.cacheFlusher.reclaimMemStoreMemory();
1653       }
1654       boolean writeToWAL = put.getWriteToWAL();
1655       region.put(put, getLockFromId(put.getLockId()), writeToWAL);
1656     } catch (Throwable t) {
1657       throw convertThrowableToIOE(cleanup(t));
1658     }
1659   }
1660 
1661   public int put(final byte[] regionName, final List<Put> puts)
1662       throws IOException {
1663     checkOpen();
1664     HRegion region = null;
1665     try {
1666       region = getRegion(regionName);
1667       if (!region.getRegionInfo().isMetaTable()) {
1668         this.cacheFlusher.reclaimMemStoreMemory();
1669       }
1670 
1671       @SuppressWarnings("unchecked")
1672       Pair<Put, Integer>[] putsWithLocks = new Pair[puts.size()];
1673 
1674       int i = 0;
1675       for (Put p : puts) {
1676         Integer lock = getLockFromId(p.getLockId());
1677         putsWithLocks[i++] = new Pair<Put, Integer>(p, lock);
1678       }
1679 
1680       this.requestCount.addAndGet(puts.size());
1681       OperationStatusCode[] codes = region.put(putsWithLocks);
1682       for (i = 0; i < codes.length; i++) {
1683         if (codes[i] != OperationStatusCode.SUCCESS) {
1684           return i;
1685         }
1686       }
1687       return -1;
1688     } catch (Throwable t) {
1689       throw convertThrowableToIOE(cleanup(t));
1690     }
1691   }
1692 
1693   private boolean checkAndMutate(final byte[] regionName, final byte[] row,
1694       final byte[] family, final byte[] qualifier, final byte[] value,
1695       final Writable w, Integer lock) throws IOException {
1696     checkOpen();
1697     this.requestCount.incrementAndGet();
1698     HRegion region = getRegion(regionName);
1699     try {
1700       if (!region.getRegionInfo().isMetaTable()) {
1701         this.cacheFlusher.reclaimMemStoreMemory();
1702       }
1703       return region
1704           .checkAndMutate(row, family, qualifier, value, w, lock, true);
1705     } catch (Throwable t) {
1706       throw convertThrowableToIOE(cleanup(t));
1707     }
1708   }
1709 
1710   /**
1711    *
1712    * @param regionName
1713    * @param row
1714    * @param family
1715    * @param qualifier
1716    * @param value
1717    *          the expected value
1718    * @param put
1719    * @throws IOException
1720    * @return true if the new put was execute, false otherwise
1721    */
1722   public boolean checkAndPut(final byte[] regionName, final byte[] row,
1723       final byte[] family, final byte[] qualifier, final byte[] value,
1724       final Put put) throws IOException {
1725     return checkAndMutate(regionName, row, family, qualifier, value, put,
1726         getLockFromId(put.getLockId()));
1727   }
1728 
1729   /**
1730    *
1731    * @param regionName
1732    * @param row
1733    * @param family
1734    * @param qualifier
1735    * @param value
1736    *          the expected value
1737    * @param delete
1738    * @throws IOException
1739    * @return true if the new put was execute, false otherwise
1740    */
1741   public boolean checkAndDelete(final byte[] regionName, final byte[] row,
1742       final byte[] family, final byte[] qualifier, final byte[] value,
1743       final Delete delete) throws IOException {
1744     return checkAndMutate(regionName, row, family, qualifier, value, delete,
1745         getLockFromId(delete.getLockId()));
1746   }
1747 
1748   //
1749   // remote scanner interface
1750   //
1751 
1752   public long openScanner(byte[] regionName, Scan scan) throws IOException {
1753     checkOpen();
1754     NullPointerException npe = null;
1755     if (regionName == null) {
1756       npe = new NullPointerException("regionName is null");
1757     } else if (scan == null) {
1758       npe = new NullPointerException("scan is null");
1759     }
1760     if (npe != null) {
1761       throw new IOException("Invalid arguments to openScanner", npe);
1762     }
1763     requestCount.incrementAndGet();
1764     try {
1765       HRegion r = getRegion(regionName);
1766       return addScanner(r.getScanner(scan));
1767     } catch (Throwable t) {
1768       throw convertThrowableToIOE(cleanup(t, "Failed openScanner"));
1769     }
1770   }
1771 
1772   protected long addScanner(InternalScanner s) throws LeaseStillHeldException {
1773     long scannerId = -1L;
1774     scannerId = rand.nextLong();
1775     String scannerName = String.valueOf(scannerId);
1776     scanners.put(scannerName, s);
1777     this.leases.createLease(scannerName, new ScannerListener(scannerName));
1778     return scannerId;
1779   }
1780 
1781   public Result next(final long scannerId) throws IOException {
1782     Result[] res = next(scannerId, 1);
1783     if (res == null || res.length == 0) {
1784       return null;
1785     }
1786     return res[0];
1787   }
1788 
1789   public Result[] next(final long scannerId, int nbRows) throws IOException {
1790     try {
1791       String scannerName = String.valueOf(scannerId);
1792       InternalScanner s = this.scanners.get(scannerName);
1793       if (s == null) {
1794         throw new UnknownScannerException("Name: " + scannerName);
1795       }
1796       try {
1797         checkOpen();
1798       } catch (IOException e) {
1799         // If checkOpen failed, server not running or filesystem gone,
1800         // cancel this lease; filesystem is gone or we're closing or something.
1801         try {
1802           this.leases.cancelLease(scannerName);
1803         } catch (LeaseException le) {
1804           LOG.info("Server shutting down and client tried to access missing scanner " +
1805             scannerName);
1806         }
1807         throw e;
1808       }
1809       this.leases.renewLease(scannerName);
1810       List<Result> results = new ArrayList<Result>(nbRows);
1811       long currentScanResultSize = 0;
1812       List<KeyValue> values = new ArrayList<KeyValue>();
1813       for (int i = 0; i < nbRows
1814           && currentScanResultSize < maxScannerResultSize; i++) {
1815         requestCount.incrementAndGet();
1816         // Collect values to be returned here
1817         boolean moreRows = s.next(values);
1818         if (!values.isEmpty()) {
1819           for (KeyValue kv : values) {
1820             currentScanResultSize += kv.heapSize();
1821           }
1822           results.add(new Result(values));
1823         }
1824         if (!moreRows) {
1825           break;
1826         }
1827         values.clear();
1828       }
1829       // Below is an ugly hack where we cast the InternalScanner to be a
1830       // HRegion.RegionScanner. The alternative is to change InternalScanner
1831       // interface but its used everywhere whereas we just need a bit of info
1832       // from HRegion.RegionScanner, IF its filter if any is done with the scan
1833       // and wants to tell the client to stop the scan. This is done by passing
1834       // a null result.
1835       return ((HRegion.RegionScanner) s).isFilterDone() && results.isEmpty() ? null
1836           : results.toArray(new Result[0]);
1837     } catch (Throwable t) {
1838       if (t instanceof NotServingRegionException) {
1839         String scannerName = String.valueOf(scannerId);
1840         this.scanners.remove(scannerName);
1841       }
1842       throw convertThrowableToIOE(cleanup(t));
1843     }
1844   }
1845 
1846   public void close(final long scannerId) throws IOException {
1847     try {
1848       checkOpen();
1849       requestCount.incrementAndGet();
1850       String scannerName = String.valueOf(scannerId);
1851       InternalScanner s = scanners.remove(scannerName);
1852       if (s != null) {
1853         s.close();
1854         this.leases.cancelLease(scannerName);
1855       }
1856     } catch (Throwable t) {
1857       throw convertThrowableToIOE(cleanup(t));
1858     }
1859   }
1860 
1861   /**
1862    * Instantiated as a scanner lease. If the lease times out, the scanner is
1863    * closed
1864    */
1865   private class ScannerListener implements LeaseListener {
1866     private final String scannerName;
1867 
1868     ScannerListener(final String n) {
1869       this.scannerName = n;
1870     }
1871 
1872     public void leaseExpired() {
1873       LOG.info("Scanner " + this.scannerName + " lease expired");
1874       InternalScanner s = scanners.remove(this.scannerName);
1875       if (s != null) {
1876         try {
1877           s.close();
1878         } catch (IOException e) {
1879           LOG.error("Closing scanner", e);
1880         }
1881       }
1882     }
1883   }
1884 
1885   //
1886   // Methods that do the actual work for the remote API
1887   //
1888   public void delete(final byte[] regionName, final Delete delete)
1889       throws IOException {
1890     checkOpen();
1891     try {
1892       boolean writeToWAL = true;
1893       this.requestCount.incrementAndGet();
1894       HRegion region = getRegion(regionName);
1895       if (!region.getRegionInfo().isMetaTable()) {
1896         this.cacheFlusher.reclaimMemStoreMemory();
1897       }
1898       Integer lid = getLockFromId(delete.getLockId());
1899       region.delete(delete, lid, writeToWAL);
1900     } catch (Throwable t) {
1901       throw convertThrowableToIOE(cleanup(t));
1902     }
1903   }
1904 
1905   public int delete(final byte[] regionName, final List<Delete> deletes)
1906       throws IOException {
1907     // Count of Deletes processed.
1908     int i = 0;
1909     checkOpen();
1910     HRegion region = null;
1911     try {
1912       boolean writeToWAL = true;
1913       region = getRegion(regionName);
1914       if (!region.getRegionInfo().isMetaTable()) {
1915         this.cacheFlusher.reclaimMemStoreMemory();
1916       }
1917       int size = deletes.size();
1918       Integer[] locks = new Integer[size];
1919       for (Delete delete : deletes) {
1920         this.requestCount.incrementAndGet();
1921         locks[i] = getLockFromId(delete.getLockId());
1922         region.delete(delete, locks[i], writeToWAL);
1923         i++;
1924       }
1925     } catch (WrongRegionException ex) {
1926       LOG.debug("Batch deletes: " + i, ex);
1927       return i;
1928     } catch (NotServingRegionException ex) {
1929       return i;
1930     } catch (Throwable t) {
1931       throw convertThrowableToIOE(cleanup(t));
1932     }
1933     return -1;
1934   }
1935 
1936   public long lockRow(byte[] regionName, byte[] row) throws IOException {
1937     checkOpen();
1938     NullPointerException npe = null;
1939     if (regionName == null) {
1940       npe = new NullPointerException("regionName is null");
1941     } else if (row == null) {
1942       npe = new NullPointerException("row to lock is null");
1943     }
1944     if (npe != null) {
1945       IOException io = new IOException("Invalid arguments to lockRow");
1946       io.initCause(npe);
1947       throw io;
1948     }
1949     requestCount.incrementAndGet();
1950     try {
1951       HRegion region = getRegion(regionName);
1952       Integer r = region.obtainRowLock(row);
1953       long lockId = addRowLock(r, region);
1954       LOG.debug("Row lock " + lockId + " explicitly acquired by client");
1955       return lockId;
1956     } catch (Throwable t) {
1957       throw convertThrowableToIOE(cleanup(t, "Error obtaining row lock (fsOk: "
1958           + this.fsOk + ")"));
1959     }
1960   }
1961 
1962   protected long addRowLock(Integer r, HRegion region)
1963       throws LeaseStillHeldException {
1964     long lockId = -1L;
1965     lockId = rand.nextLong();
1966     String lockName = String.valueOf(lockId);
1967     rowlocks.put(lockName, r);
1968     this.leases.createLease(lockName, new RowLockListener(lockName, region));
1969     return lockId;
1970   }
1971 
1972   /**
1973    * Method to get the Integer lock identifier used internally from the long
1974    * lock identifier used by the client.
1975    *
1976    * @param lockId
1977    *          long row lock identifier from client
1978    * @return intId Integer row lock used internally in HRegion
1979    * @throws IOException
1980    *           Thrown if this is not a valid client lock id.
1981    */
1982   Integer getLockFromId(long lockId) throws IOException {
1983     if (lockId == -1L) {
1984       return null;
1985     }
1986     String lockName = String.valueOf(lockId);
1987     Integer rl = rowlocks.get(lockName);
1988     if (rl == null) {
1989       throw new UnknownRowLockException("Invalid row lock");
1990     }
1991     this.leases.renewLease(lockName);
1992     return rl;
1993   }
1994 
1995   @Override
1996   @QosPriority(priority=HIGH_QOS)
1997   public void unlockRow(byte[] regionName, long lockId) throws IOException {
1998     checkOpen();
1999     NullPointerException npe = null;
2000     if (regionName == null) {
2001       npe = new NullPointerException("regionName is null");
2002     } else if (lockId == -1L) {
2003       npe = new NullPointerException("lockId is null");
2004     }
2005     if (npe != null) {
2006       IOException io = new IOException("Invalid arguments to unlockRow");
2007       io.initCause(npe);
2008       throw io;
2009     }
2010     requestCount.incrementAndGet();
2011     try {
2012       HRegion region = getRegion(regionName);
2013       String lockName = String.valueOf(lockId);
2014       Integer r = rowlocks.remove(lockName);
2015       if (r == null) {
2016         throw new UnknownRowLockException(lockName);
2017       }
2018       region.releaseRowLock(r);
2019       this.leases.cancelLease(lockName);
2020       LOG.debug("Row lock " + lockId
2021           + " has been explicitly released by client");
2022     } catch (Throwable t) {
2023       throw convertThrowableToIOE(cleanup(t));
2024     }
2025   }
2026 
2027   @Override
2028   public void bulkLoadHFile(String hfilePath, byte[] regionName,
2029       byte[] familyName) throws IOException {
2030     HRegion region = getRegion(regionName);
2031     region.bulkLoadHFile(hfilePath, familyName);
2032   }
2033 
2034   Map<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
2035 
2036   /**
2037    * Instantiated as a row lock lease. If the lease times out, the row lock is
2038    * released
2039    */
2040   private class RowLockListener implements LeaseListener {
2041     private final String lockName;
2042     private final HRegion region;
2043 
2044     RowLockListener(final String lockName, final HRegion region) {
2045       this.lockName = lockName;
2046       this.region = region;
2047     }
2048 
2049     public void leaseExpired() {
2050       LOG.info("Row Lock " + this.lockName + " lease expired");
2051       Integer r = rowlocks.remove(this.lockName);
2052       if (r != null) {
2053         region.releaseRowLock(r);
2054       }
2055     }
2056   }
2057 
2058   // Region open/close direct RPCs
2059 
2060   @Override
2061   @QosPriority(priority=HIGH_QOS)
2062   public void openRegion(HRegionInfo region)
2063   throws RegionServerStoppedException {
2064     LOG.info("Received request to open region: " +
2065       region.getRegionNameAsString());
2066     if (this.stopped) throw new RegionServerStoppedException();
2067     if (region.isRootRegion()) {
2068       this.service.submit(new OpenRootHandler(this, this, region));
2069     } else if(region.isMetaRegion()) {
2070       this.service.submit(new OpenMetaHandler(this, this, region));
2071     } else {
2072       this.service.submit(new OpenRegionHandler(this, this, region));
2073     }
2074   }
2075 
2076   @Override
2077   @QosPriority(priority=HIGH_QOS)
2078   public void openRegions(List<HRegionInfo> regions)
2079   throws RegionServerStoppedException {
2080     LOG.info("Received request to open " + regions.size() + " region(s)");
2081     for (HRegionInfo region: regions) openRegion(region);
2082   }
2083 
2084   @Override
2085   @QosPriority(priority=HIGH_QOS)
2086   public boolean closeRegion(HRegionInfo region)
2087   throws NotServingRegionException {
2088     return closeRegion(region, true);
2089   }
2090 
2091   @Override
2092   @QosPriority(priority=HIGH_QOS)
2093   public boolean closeRegion(HRegionInfo region, final boolean zk)
2094   throws NotServingRegionException {
2095     LOG.info("Received close region: " + region.getRegionNameAsString());
2096     boolean hasit = this.onlineRegions.containsKey(region.getEncodedName());
2097     if (!hasit) {
2098       LOG.warn("Received close for region we are not serving; " +
2099         region.getEncodedName());
2100       throw new NotServingRegionException("Received close for "
2101         + region.getRegionNameAsString() + " but we are not serving it");
2102     }
2103     return closeRegion(region, false, zk);
2104   }
2105 
2106   /**
2107    * @param region Region to close
2108    * @param abort True if we are aborting
2109    * @param zk True if we are to update zk about the region close; if the close
2110    * was orchestrated by master, then update zk.  If the close is being run by
2111    * the regionserver because its going down, don't update zk.
2112    * @return True if closed a region.
2113    */
2114   protected boolean closeRegion(HRegionInfo region, final boolean abort,
2115       final boolean zk) {
2116     CloseRegionHandler crh = null;
2117     if (region.isRootRegion()) {
2118       crh = new CloseRootHandler(this, this, region, abort, zk);
2119     } else if (region.isMetaRegion()) {
2120       crh = new CloseMetaHandler(this, this, region, abort, zk);
2121     } else {
2122       crh = new CloseRegionHandler(this, this, region, abort, zk);
2123     }
2124     this.service.submit(crh);
2125     return true;
2126   }
2127 
2128   // Manual remote region administration RPCs
2129 
2130   @Override
2131   @QosPriority(priority=HIGH_QOS)
2132   public void flushRegion(HRegionInfo regionInfo)
2133       throws NotServingRegionException, IOException {
2134     LOG.info("Flushing " + regionInfo.getRegionNameAsString());
2135     HRegion region = getRegion(regionInfo.getRegionName());
2136     region.flushcache();
2137   }
2138 
2139   @Override
2140   @QosPriority(priority=HIGH_QOS)
2141   public void splitRegion(HRegionInfo regionInfo)
2142       throws NotServingRegionException, IOException {
2143     splitRegion(regionInfo, null);
2144   }
2145 
2146   @Override
2147   public void splitRegion(HRegionInfo regionInfo, byte[] splitPoint)
2148       throws NotServingRegionException, IOException {
2149     HRegion region = getRegion(regionInfo.getRegionName());
2150     region.flushcache();
2151     region.forceSplit(splitPoint);
2152     // force a compaction, split will be side-effect
2153     // TODO: flush/compact/split refactor will make it trivial to do this
2154     // sync/async (and won't require us to do a compaction to split!)
2155     compactSplitThread.requestCompaction(region, "User-triggered split",
2156         CompactSplitThread.PRIORITY_USER);
2157   }
2158 
2159   @Override
2160   @QosPriority(priority=HIGH_QOS)
2161   public void compactRegion(HRegionInfo regionInfo, boolean major)
2162       throws NotServingRegionException, IOException {
2163     HRegion region = getRegion(regionInfo.getRegionName());
2164     compactSplitThread.requestCompaction(region, major, "User-triggered "
2165         + (major ? "major " : "") + "compaction",
2166         CompactSplitThread.PRIORITY_USER);
2167   }
2168 
2169   /** @return the info server */
2170   public InfoServer getInfoServer() {
2171     return infoServer;
2172   }
2173 
2174   /**
2175    * @return true if a stop has been requested.
2176    */
2177   public boolean isStopped() {
2178     return this.stopped;
2179   }
2180 
2181   @Override
2182   public boolean isStopping() {
2183     return this.stopping;
2184   }
2185 
2186   /**
2187    *
2188    * @return the configuration
2189    */
2190   public Configuration getConfiguration() {
2191     return conf;
2192   }
2193 
2194   /** @return the write lock for the server */
2195   ReentrantReadWriteLock.WriteLock getWriteLock() {
2196     return lock.writeLock();
2197   }
2198 
2199   @Override
2200   @QosPriority(priority=HIGH_QOS)
2201   public List<HRegionInfo> getOnlineRegions() {
2202     List<HRegionInfo> list = new ArrayList<HRegionInfo>(onlineRegions.size());
2203     for (Map.Entry<String,HRegion> e: this.onlineRegions.entrySet()) {
2204       list.add(e.getValue().getRegionInfo());
2205     }
2206     Collections.sort(list);
2207     return list;
2208   }
2209 
2210   public int getNumberOfOnlineRegions() {
2211     int size = -1;
2212     size = this.onlineRegions.size();
2213     return size;
2214   }
2215 
2216   boolean isOnlineRegionsEmpty() {
2217     return this.onlineRegions.isEmpty();
2218   }
2219 
2220   /**
2221    * For tests and web ui.
2222    * This method will only work if HRegionServer is in the same JVM as client;
2223    * HRegion cannot be serialized to cross an rpc.
2224    * @see #getOnlineRegions()
2225    */
2226   public Collection<HRegion> getOnlineRegionsLocalContext() {
2227     Collection<HRegion> regions = this.onlineRegions.values();
2228     return Collections.unmodifiableCollection(regions);
2229   }
2230 
2231   @Override
2232   public void addToOnlineRegions(HRegion region) {
2233     this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
2234   }
2235 
2236   @Override
2237   public boolean removeFromOnlineRegions(final String encodedName) {
2238     HRegion toReturn = null;
2239     toReturn = this.onlineRegions.remove(encodedName);
2240     return toReturn != null;
2241   }
2242 
2243   /**
2244    * @return A new Map of online regions sorted by region size with the first
2245    *         entry being the biggest.
2246    */
2247   public SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
2248     // we'll sort the regions in reverse
2249     SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
2250         new Comparator<Long>() {
2251           public int compare(Long a, Long b) {
2252             return -1 * a.compareTo(b);
2253           }
2254         });
2255     // Copy over all regions. Regions are sorted by size with biggest first.
2256     for (HRegion region : this.onlineRegions.values()) {
2257       sortedRegions.put(Long.valueOf(region.memstoreSize.get()), region);
2258     }
2259     return sortedRegions;
2260   }
2261 
2262   @Override
2263   public HRegion getFromOnlineRegions(final String encodedRegionName) {
2264     HRegion r = null;
2265     r = this.onlineRegions.get(encodedRegionName);
2266     return r;
2267   }
2268 
2269   /**
2270    * @param regionName
2271    * @return HRegion for the passed binary <code>regionName</code> or null if
2272    *         named region is not member of the online regions.
2273    */
2274   public HRegion getOnlineRegion(final byte[] regionName) {
2275     return getFromOnlineRegions(HRegionInfo.encodeRegionName(regionName));
2276   }
2277 
2278   /** @return the request count */
2279   public AtomicInteger getRequestCount() {
2280     return this.requestCount;
2281   }
2282 
2283   /** @return reference to FlushRequester */
2284   public FlushRequester getFlushRequester() {
2285     return this.cacheFlusher;
2286   }
2287 
2288   /**
2289    * Protected utility method for safely obtaining an HRegion handle.
2290    *
2291    * @param regionName
2292    *          Name of online {@link HRegion} to return
2293    * @return {@link HRegion} for <code>regionName</code>
2294    * @throws NotServingRegionException
2295    */
2296   protected HRegion getRegion(final byte[] regionName)
2297       throws NotServingRegionException {
2298     HRegion region = null;
2299     region = getOnlineRegion(regionName);
2300     if (region == null) {
2301       throw new NotServingRegionException("Region is not online: " +
2302         Bytes.toStringBinary(regionName));
2303     }
2304     return region;
2305   }
2306 
2307   /**
2308    * Get the top N most loaded regions this server is serving so we can tell the
2309    * master which regions it can reallocate if we're overloaded. TODO: actually
2310    * calculate which regions are most loaded. (Right now, we're just grabbing
2311    * the first N regions being served regardless of load.)
2312    */
2313   protected HRegionInfo[] getMostLoadedRegions() {
2314     ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
2315     for (HRegion r : onlineRegions.values()) {
2316       if (r.isClosed() || r.isClosing()) {
2317         continue;
2318       }
2319       if (regions.size() < numRegionsToReport) {
2320         regions.add(r.getRegionInfo());
2321       } else {
2322         break;
2323       }
2324     }
2325     return regions.toArray(new HRegionInfo[regions.size()]);
2326   }
2327 
2328   /**
2329    * Called to verify that this server is up and running.
2330    *
2331    * @throws IOException
2332    */
2333   protected void checkOpen() throws IOException {
2334     if (this.stopped || this.abortRequested) {
2335       throw new IOException("Server not running"
2336           + (this.abortRequested ? ", aborting" : ""));
2337     }
2338     if (!fsOk) {
2339       throw new IOException("File system not available");
2340     }
2341   }
2342 
2343   /**
2344    * @return Returns list of non-closed regions hosted on this server. If no
2345    *         regions to check, returns an empty list.
2346    */
2347   protected Set<HRegion> getRegionsToCheck() {
2348     HashSet<HRegion> regionsToCheck = new HashSet<HRegion>();
2349     regionsToCheck.addAll(this.onlineRegions.values());
2350     // Purge closed regions.
2351     for (final Iterator<HRegion> i = regionsToCheck.iterator(); i.hasNext();) {
2352       HRegion r = i.next();
2353       if (r.isClosed()) {
2354         i.remove();
2355       }
2356     }
2357     return regionsToCheck;
2358   }
2359 
2360   @Override
2361   @QosPriority(priority=HIGH_QOS)
2362   public long getProtocolVersion(final String protocol, final long clientVersion)
2363       throws IOException {
2364     if (protocol.equals(HRegionInterface.class.getName())) {
2365       return HBaseRPCProtocolVersion.versionID;
2366     }
2367     throw new IOException("Unknown protocol to name node: " + protocol);
2368   }
2369 
2370   /**
2371    * @return Queue to which you can add outbound messages.
2372    */
2373   protected LinkedBlockingQueue<HMsg> getOutboundMsgs() {
2374     return this.outboundMsgs;
2375   }
2376 
2377   /**
2378    * Return the total size of all memstores in every region.
2379    *
2380    * @return memstore size in bytes
2381    */
2382   public long getGlobalMemStoreSize() {
2383     long total = 0;
2384     for (HRegion region : onlineRegions.values()) {
2385       total += region.memstoreSize.get();
2386     }
2387     return total;
2388   }
2389 
2390   /**
2391    * @return Return the leases.
2392    */
2393   protected Leases getLeases() {
2394     return leases;
2395   }
2396 
2397   /**
2398    * @return Return the rootDir.
2399    */
2400   protected Path getRootDir() {
2401     return rootDir;
2402   }
2403 
2404   /**
2405    * @return Return the fs.
2406    */
2407   protected FileSystem getFileSystem() {
2408     return fs;
2409   }
2410 
2411   /**
2412    * @return Info on port this server has bound to, etc.
2413    */
2414   public HServerInfo getServerInfo() {
2415     return this.serverInfo;
2416   }
2417 
2418 
2419   @Override
2420   public Result increment(byte[] regionName, Increment increment)
2421   throws IOException {
2422     checkOpen();
2423     if (regionName == null) {
2424       throw new IOException("Invalid arguments to increment " +
2425       "regionName is null");
2426     }
2427     requestCount.incrementAndGet();
2428     try {
2429       HRegion region = getRegion(regionName);
2430       return region.increment(increment, getLockFromId(increment.getLockId()),
2431           increment.getWriteToWAL());
2432     } catch (IOException e) {
2433       checkFileSystem();
2434       throw e;
2435     }
2436   }
2437 
2438   /** {@inheritDoc} */
2439   public long incrementColumnValue(byte[] regionName, byte[] row,
2440       byte[] family, byte[] qualifier, long amount, boolean writeToWAL)
2441       throws IOException {
2442     checkOpen();
2443 
2444     if (regionName == null) {
2445       throw new IOException("Invalid arguments to incrementColumnValue "
2446           + "regionName is null");
2447     }
2448     requestCount.incrementAndGet();
2449     try {
2450       HRegion region = getRegion(regionName);
2451       long retval = region.incrementColumnValue(row, family, qualifier, amount,
2452           writeToWAL);
2453 
2454       return retval;
2455     } catch (IOException e) {
2456       checkFileSystem();
2457       throw e;
2458     }
2459   }
2460 
2461   /** {@inheritDoc} */
2462   @Override
2463   @QosPriority(priority=HIGH_QOS)
2464   public HServerInfo getHServerInfo() throws IOException {
2465     return serverInfo;
2466   }
2467 
2468   @SuppressWarnings("unchecked")
2469   @Override
2470   public MultiResponse multi(MultiAction multi) throws IOException {
2471 
2472     MultiResponse response = new MultiResponse();
2473 
2474     for (Map.Entry<byte[], List<Action>> e : multi.actions.entrySet()) {
2475       byte[] regionName = e.getKey();
2476       List<Action> actionsForRegion = e.getValue();
2477       // sort based on the row id - this helps in the case where we reach the
2478       // end of a region, so that we don't have to try the rest of the
2479       // actions in the list.
2480       Collections.sort(actionsForRegion);
2481       Row action;
2482       List<Action> puts = new ArrayList<Action>();
2483       for (Action a : actionsForRegion) {
2484         action = a.getAction();
2485         int originalIndex = a.getOriginalIndex();
2486 
2487         try {
2488           if (action instanceof Delete) {
2489             delete(regionName, (Delete) action);
2490             response.add(regionName, originalIndex, new Result());
2491           } else if (action instanceof Get) {
2492             response.add(regionName, originalIndex, get(regionName, (Get) action));
2493           } else if (action instanceof Put) {
2494             puts.add(a);  // wont throw.
2495           } else {
2496             LOG.debug("Error: invalid Action, row must be a Get, Delete or Put.");
2497             throw new DoNotRetryIOException("Invalid Action, row must be a Get, Delete or Put.");
2498           }
2499         } catch (IOException ex) {
2500           response.add(regionName, originalIndex, ex);
2501         }
2502       }
2503 
2504       // We do the puts with result.put so we can get the batching efficiency
2505       // we so need. All this data munging doesn't seem great, but at least
2506       // we arent copying bytes or anything.
2507       if (!puts.isEmpty()) {
2508         try {
2509           HRegion region = getRegion(regionName);
2510 
2511           if (!region.getRegionInfo().isMetaTable()) {
2512             this.cacheFlusher.reclaimMemStoreMemory();
2513           }
2514 
2515           List<Pair<Put,Integer>> putsWithLocks =
2516               Lists.newArrayListWithCapacity(puts.size());
2517           for (Action a : puts) {
2518             Put p = (Put) a.getAction();
2519 
2520             Integer lock;
2521             try {
2522               lock = getLockFromId(p.getLockId());
2523             } catch (UnknownRowLockException ex) {
2524               response.add(regionName, a.getOriginalIndex(), ex);
2525               continue;
2526             }
2527             putsWithLocks.add(new Pair<Put, Integer>(p, lock));
2528           }
2529 
2530           this.requestCount.addAndGet(puts.size());
2531 
2532           OperationStatusCode[] codes =
2533               region.put(putsWithLocks.toArray(new Pair[]{}));
2534 
2535           for( int i = 0 ; i < codes.length ; i++) {
2536             OperationStatusCode code = codes[i];
2537 
2538             Action theAction = puts.get(i);
2539             Object result = null;
2540 
2541             if (code == OperationStatusCode.SUCCESS) {
2542               result = new Result();
2543             } else if (code == OperationStatusCode.BAD_FAMILY) {
2544               result = new NoSuchColumnFamilyException();
2545             }
2546             // FAILURE && NOT_RUN becomes null, aka: need to run again.
2547 
2548             response.add(regionName, theAction.getOriginalIndex(), result);
2549           }
2550         } catch (IOException ioe) {
2551           // fail all the puts with the ioe in question.
2552           for (Action a: puts) {
2553             response.add(regionName, a.getOriginalIndex(), ioe);
2554           }
2555         }
2556       }
2557     }
2558     return response;
2559   }
2560 
2561   /**
2562    * @deprecated Use HRegionServer.multi( MultiAction action) instead
2563    */
2564   @Override
2565   public MultiPutResponse multiPut(MultiPut puts) throws IOException {
2566     MultiPutResponse resp = new MultiPutResponse();
2567 
2568     // do each region as it's own.
2569     for (Map.Entry<byte[], List<Put>> e : puts.puts.entrySet()) {
2570       int result = put(e.getKey(), e.getValue());
2571       resp.addResult(e.getKey(), result);
2572 
2573       e.getValue().clear(); // clear some RAM
2574     }
2575 
2576     return resp;
2577   }
2578 
2579   public String toString() {
2580     return this.serverInfo.toString();
2581   }
2582 
2583   /**
2584    * Interval at which threads should run
2585    *
2586    * @return the interval
2587    */
2588   public int getThreadWakeFrequency() {
2589     return threadWakeFrequency;
2590   }
2591 
2592   @Override
2593   public ZooKeeperWatcher getZooKeeper() {
2594     return zooKeeper;
2595   }
2596 
2597   @Override
2598   public String getServerName() {
2599     return serverInfo.getServerName();
2600   }
2601 
2602   @Override
2603   public CompactionRequestor getCompactionRequester() {
2604     return this.compactSplitThread;
2605   }
2606 
2607   //
2608   // Main program and support routines
2609   //
2610 
2611   /**
2612    * @param hrs
2613    * @return Thread the RegionServer is running in correctly named.
2614    * @throws IOException
2615    */
2616   public static Thread startRegionServer(final HRegionServer hrs)
2617       throws IOException {
2618     return startRegionServer(hrs, "regionserver"
2619         + hrs.getServerInfo().getServerAddress().getPort());
2620   }
2621 
2622   /**
2623    * @param hrs
2624    * @param name
2625    * @return Thread the RegionServer is running in correctly named.
2626    * @throws IOException
2627    */
2628   public static Thread startRegionServer(final HRegionServer hrs,
2629       final String name) throws IOException {
2630     Thread t = new Thread(hrs);
2631     t.setName(name);
2632     t.start();
2633     // Install shutdown hook that will catch signals and run an orderly shutdown
2634     // of the hrs.
2635     ShutdownHook.install(hrs.getConfiguration(), FileSystem.get(hrs
2636         .getConfiguration()), hrs, t);
2637     return t;
2638   }
2639 
2640   /**
2641    * Utility for constructing an instance of the passed HRegionServer class.
2642    *
2643    * @param regionServerClass
2644    * @param conf2
2645    * @return HRegionServer instance.
2646    */
2647   public static HRegionServer constructRegionServer(
2648       Class<? extends HRegionServer> regionServerClass,
2649       final Configuration conf2) {
2650     try {
2651       Constructor<? extends HRegionServer> c = regionServerClass
2652           .getConstructor(Configuration.class);
2653       return c.newInstance(conf2);
2654     } catch (Exception e) {
2655       throw new RuntimeException("Failed construction of " + "Regionserver: "
2656           + regionServerClass.toString(), e);
2657     }
2658   }
2659 
2660   @Override
2661   public void replicateLogEntries(final HLog.Entry[] entries)
2662   throws IOException {
2663     if (this.replicationHandler == null) return;
2664     this.replicationHandler.replicateLogEntries(entries);
2665   }
2666 
2667 
2668   /**
2669    * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
2670    */
2671   public static void main(String[] args) throws Exception {
2672     Configuration conf = HBaseConfiguration.create();
2673     @SuppressWarnings("unchecked")
2674     Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
2675         .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
2676 
2677     new HRegionServerCommandLine(regionServerClass).doMain(args);
2678   }
2679 }