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