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