View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  import java.io.InterruptedIOException;
23  import java.lang.Thread.UncaughtExceptionHandler;
24  import java.lang.management.ManagementFactory;
25  import java.lang.management.MemoryUsage;
26  import java.lang.reflect.Constructor;
27  import java.net.BindException;
28  import java.net.InetSocketAddress;
29  import java.util.ArrayList;
30  import java.util.Collection;
31  import java.util.Collections;
32  import java.util.Comparator;
33  import java.util.HashMap;
34  import java.util.HashSet;
35  import java.util.Iterator;
36  import java.util.List;
37  import java.util.Map;
38  import java.util.Map.Entry;
39  import java.util.Set;
40  import java.util.SortedMap;
41  import java.util.TreeMap;
42  import java.util.TreeSet;
43  import java.util.concurrent.ConcurrentHashMap;
44  import java.util.concurrent.ConcurrentMap;
45  import java.util.concurrent.ConcurrentSkipListMap;
46  import java.util.concurrent.atomic.AtomicBoolean;
47  import java.util.concurrent.locks.ReentrantReadWriteLock;
48  
49  import javax.management.ObjectName;
50  import javax.servlet.http.HttpServlet;
51  
52  import org.apache.commons.lang.math.RandomUtils;
53  import org.apache.commons.logging.Log;
54  import org.apache.commons.logging.LogFactory;
55  import org.apache.hadoop.classification.InterfaceAudience;
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.CoordinatedStateManager;
62  import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
63  import org.apache.hadoop.hbase.HBaseConfiguration;
64  import org.apache.hadoop.hbase.HConstants;
65  import org.apache.hadoop.hbase.HRegionInfo;
66  import org.apache.hadoop.hbase.HealthCheckChore;
67  import org.apache.hadoop.hbase.MetaTableAccessor;
68  import org.apache.hadoop.hbase.NotServingRegionException;
69  import org.apache.hadoop.hbase.ServerName;
70  import org.apache.hadoop.hbase.Stoppable;
71  import org.apache.hadoop.hbase.TableDescriptors;
72  import org.apache.hadoop.hbase.TableName;
73  import org.apache.hadoop.hbase.YouAreDeadException;
74  import org.apache.hadoop.hbase.ZNodeClearer;
75  import org.apache.hadoop.hbase.client.ConnectionUtils;
76  import org.apache.hadoop.hbase.client.HConnection;
77  import org.apache.hadoop.hbase.client.HConnectionManager;
78  import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
79  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
80  import org.apache.hadoop.hbase.exceptions.RegionMovedException;
81  import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
82  import org.apache.hadoop.hbase.executor.ExecutorService;
83  import org.apache.hadoop.hbase.executor.ExecutorType;
84  import org.apache.hadoop.hbase.fs.HFileSystem;
85  import org.apache.hadoop.hbase.http.InfoServer;
86  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
87  import org.apache.hadoop.hbase.ipc.RpcClient;
88  import org.apache.hadoop.hbase.ipc.RpcServerInterface;
89  import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
90  import org.apache.hadoop.hbase.master.HMaster;
91  import org.apache.hadoop.hbase.master.SplitLogManager;
92  import org.apache.hadoop.hbase.master.TableLockManager;
93  import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
94  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
95  import org.apache.hadoop.hbase.protobuf.RequestConverter;
96  import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
97  import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
98  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
99  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
100 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
101 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
102 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
103 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
104 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
105 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
106 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
107 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
108 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
109 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
110 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
111 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
112 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
113 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
114 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
115 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
116 import org.apache.hadoop.hbase.regionserver.wal.HLog;
117 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
118 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
119 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
120 import org.apache.hadoop.hbase.security.UserProvider;
121 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
122 import org.apache.hadoop.hbase.util.ByteStringer;
123 import org.apache.hadoop.hbase.util.Bytes;
124 import org.apache.hadoop.hbase.util.CompressionTest;
125 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
126 import org.apache.hadoop.hbase.util.FSTableDescriptors;
127 import org.apache.hadoop.hbase.util.FSUtils;
128 import org.apache.hadoop.hbase.util.HasThread;
129 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
130 import org.apache.hadoop.hbase.util.Sleeper;
131 import org.apache.hadoop.hbase.util.Threads;
132 import org.apache.hadoop.hbase.util.VersionInfo;
133 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
134 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
135 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
136 import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher;
137 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
138 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
139 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
140 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
141 import org.apache.hadoop.ipc.RemoteException;
142 import org.apache.hadoop.metrics.util.MBeanUtil;
143 import org.apache.hadoop.util.ReflectionUtils;
144 import org.apache.hadoop.util.StringUtils;
145 import org.apache.zookeeper.KeeperException;
146 import org.apache.zookeeper.KeeperException.NoNodeException;
147 import org.apache.zookeeper.data.Stat;
148 
149 import com.google.common.annotations.VisibleForTesting;
150 import com.google.common.base.Preconditions;
151 import com.google.protobuf.BlockingRpcChannel;
152 import com.google.protobuf.ServiceException;
153 
154 /**
155  * HRegionServer makes a set of HRegions available to clients. It checks in with
156  * the HMaster. There are many HRegionServers in a single HBase deployment.
157  */
158 @InterfaceAudience.Private
159 @SuppressWarnings("deprecation")
160 public class HRegionServer extends HasThread implements
161     RegionServerServices, LastSequenceId {
162 
163   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
164 
165   /**
166    * For testing only!  Set to true to skip notifying region assignment to master .
167    */
168   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
169   public static boolean TEST_SKIP_REPORTING_TRANSITION = false;
170 
171   /*
172    * Strings to be used in forming the exception message for
173    * RegionsAlreadyInTransitionException.
174    */
175   protected static final String OPEN = "OPEN";
176   protected static final String CLOSE = "CLOSE";
177 
178   //RegionName vs current action in progress
179   //true - if open region action in progress
180   //false - if close region action in progress
181   protected final ConcurrentMap<byte[], Boolean> regionsInTransitionInRS =
182     new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
183 
184   // Cache flushing
185   protected MemStoreFlusher cacheFlusher;
186 
187   protected HeapMemoryManager hMemManager;
188 
189   /*
190    * Short-circuit (ie. bypassing RPC layer) HConnection to this Server
191    * to be used internally for miscellaneous needs. Initialized at the server startup
192    * and closed when server shuts down. Clients must never close it explicitly.
193    */
194   protected HConnection shortCircuitConnection;
195 
196   /*
197    * Long-living meta table locator, which is created when the server is started and stopped
198    * when server shuts down. References to this locator shall be used to perform according
199    * operations in EventHandlers. Primary reason for this decision is to make it mockable
200    * for tests.
201    */
202   protected MetaTableLocator metaTableLocator;
203 
204   // Watch if a region is out of recovering state from ZooKeeper
205   @SuppressWarnings("unused")
206   private RecoveringRegionWatcher recoveringRegionWatcher;
207 
208   /**
209    * Go here to get table descriptors.
210    */
211   protected TableDescriptors tableDescriptors;
212 
213   // Replication services. If no replication, this handler will be null.
214   protected ReplicationSourceService replicationSourceHandler;
215   protected ReplicationSinkService replicationSinkHandler;
216 
217   // Compactions
218   public CompactSplitThread compactSplitThread;
219 
220   /**
221    * Map of regions currently being served by this region server. Key is the
222    * encoded region name.  All access should be synchronized.
223    */
224   protected final Map<String, HRegion> onlineRegions =
225     new ConcurrentHashMap<String, HRegion>();
226 
227   /**
228    * Map of encoded region names to the DataNode locations they should be hosted on
229    * We store the value as InetSocketAddress since this is used only in HDFS
230    * API (create() that takes favored nodes as hints for placing file blocks).
231    * We could have used ServerName here as the value class, but we'd need to
232    * convert it to InetSocketAddress at some point before the HDFS API call, and
233    * it seems a bit weird to store ServerName since ServerName refers to RegionServers
234    * and here we really mean DataNode locations.
235    */
236   protected final Map<String, InetSocketAddress[]> regionFavoredNodesMap =
237       new ConcurrentHashMap<String, InetSocketAddress[]>();
238 
239   /**
240    * Set of regions currently being in recovering state which means it can accept writes(edits from
241    * previous failed region server) but not reads. A recovering region is also an online region.
242    */
243   protected final Map<String, HRegion> recoveringRegions = Collections
244       .synchronizedMap(new HashMap<String, HRegion>());
245 
246   // Leases
247   protected Leases leases;
248 
249   // Instance of the hbase executor service.
250   protected ExecutorService service;
251 
252   // If false, the file system has become unavailable
253   protected volatile boolean fsOk;
254   protected HFileSystem fs;
255 
256   // Set when a report to the master comes back with a message asking us to
257   // shutdown. Also set by call to stop when debugging or running unit tests
258   // of HRegionServer in isolation.
259   private volatile boolean stopped = false;
260 
261   // Go down hard. Used if file system becomes unavailable and also in
262   // debugging and unit tests.
263   private volatile boolean abortRequested;
264 
265   ConcurrentMap<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
266 
267   // A state before we go into stopped state.  At this stage we're closing user
268   // space regions.
269   private boolean stopping = false;
270 
271   private volatile boolean killed = false;
272 
273   protected final Configuration conf;
274 
275   private Path rootDir;
276 
277   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
278 
279   final int numRetries;
280   protected final int threadWakeFrequency;
281   protected final int msgInterval;
282 
283   protected final int numRegionsToReport;
284 
285   // Stub to do region server status calls against the master.
286   private volatile RegionServerStatusService.BlockingInterface rssStub;
287   // RPC client. Used to make the stub above that does region server status checking.
288   RpcClient rpcClient;
289 
290   private UncaughtExceptionHandler uncaughtExceptionHandler;
291 
292   // Info server. Default access so can be used by unit tests. REGIONSERVER
293   // is name of the webapp and the attribute name used stuffing this instance
294   // into web context.
295   protected InfoServer infoServer;
296   private JvmPauseMonitor pauseMonitor;
297 
298   /** region server process name */
299   public static final String REGIONSERVER = "regionserver";
300 
301   MetricsRegionServer metricsRegionServer;
302   private SpanReceiverHost spanReceiverHost;
303 
304   /*
305    * Check for compactions requests.
306    */
307   Chore compactionChecker;
308 
309   /*
310    * Check for flushes
311    */
312   Chore periodicFlusher;
313 
314   // HLog and HLog roller. log is protected rather than private to avoid
315   // eclipse warning when accessed by inner classes
316   protected volatile HLog hlog;
317   // The meta updates are written to a different hlog. If this
318   // regionserver holds meta regions, then this field will be non-null.
319   protected volatile HLog hlogForMeta;
320 
321   LogRoller hlogRoller;
322   LogRoller metaHLogRoller;
323 
324   // flag set after we're done setting up server threads
325   protected AtomicBoolean online;
326 
327   // zookeeper connection and watcher
328   protected ZooKeeperWatcher zooKeeper;
329 
330   // master address tracker
331   private MasterAddressTracker masterAddressTracker;
332 
333   // Cluster Status Tracker
334   protected ClusterStatusTracker clusterStatusTracker;
335 
336   // Log Splitting Worker
337   private SplitLogWorker splitLogWorker;
338 
339   // A sleeper that sleeps for msgInterval.
340   protected final Sleeper sleeper;
341 
342   private final int operationTimeout;
343 
344   private final RegionServerAccounting regionServerAccounting;
345 
346   // Cache configuration and block cache reference
347   final CacheConfig cacheConfig;
348 
349   /** The health check chore. */
350   private HealthCheckChore healthCheckChore;
351 
352   /** The nonce manager chore. */
353   private Chore nonceManagerChore;
354 
355   /**
356    * The server name the Master sees us as.  Its made from the hostname the
357    * master passes us, port, and server startcode. Gets set after registration
358    * against  Master.
359    */
360   protected ServerName serverName;
361 
362   /**
363    * This servers startcode.
364    */
365   protected final long startcode;
366 
367   /**
368    * Unique identifier for the cluster we are a part of.
369    */
370   private String clusterId;
371 
372   /**
373    * MX Bean for RegionServerInfo
374    */
375   private ObjectName mxBean = null;
376 
377   /**
378    * Chore to clean periodically the moved region list
379    */
380   private MovedRegionsCleaner movedRegionsCleaner;
381 
382   // chore for refreshing store files for secondary regions
383   private StorefileRefresherChore storefileRefresher;
384 
385   private RegionServerCoprocessorHost rsHost;
386 
387   private RegionServerProcedureManagerHost rspmHost;
388 
389   // Table level lock manager for locking for region operations
390   protected TableLockManager tableLockManager;
391 
392   /**
393    * Nonce manager. Nonces are used to make operations like increment and append idempotent
394    * in the case where client doesn't receive the response from a successful operation and
395    * retries. We track the successful ops for some time via a nonce sent by client and handle
396    * duplicate operations (currently, by failing them; in future we might use MVCC to return
397    * result). Nonces are also recovered from WAL during, recovery; however, the caveats (from
398    * HBASE-3787) are:
399    * - WAL recovery is optimized, and under high load we won't read nearly nonce-timeout worth
400    *   of past records. If we don't read the records, we don't read and recover the nonces.
401    *   Some WALs within nonce-timeout at recovery may not even be present due to rolling/cleanup.
402    * - There's no WAL recovery during normal region move, so nonces will not be transfered.
403    * We can have separate additional "Nonce WAL". It will just contain bunch of numbers and
404    * won't be flushed on main path - because WAL itself also contains nonces, if we only flush
405    * it before memstore flush, for a given nonce we will either see it in the WAL (if it was
406    * never flushed to disk, it will be part of recovery), or we'll see it as part of the nonce
407    * log (or both occasionally, which doesn't matter). Nonce log file can be deleted after the
408    * latest nonce in it expired. It can also be recovered during move.
409    */
410   final ServerNonceManager nonceManager;
411 
412   private UserProvider userProvider;
413 
414   protected final RSRpcServices rpcServices;
415 
416   protected BaseCoordinatedStateManager csm;
417 
418   /**
419    * Starts a HRegionServer at the default location.
420    * @param conf
421    * @throws IOException
422    * @throws InterruptedException
423    */
424   public HRegionServer(Configuration conf) throws IOException, InterruptedException {
425     this(conf, CoordinatedStateManagerFactory.getCoordinatedStateManager(conf));
426   }
427 
428   /**
429    * Starts a HRegionServer at the default location
430    * @param conf
431    * @param csm implementation of CoordinatedStateManager to be used
432    * @throws IOException
433    */
434   public HRegionServer(Configuration conf, CoordinatedStateManager csm)
435       throws IOException {
436     this.fsOk = true;
437     this.conf = conf;
438     checkCodecs(this.conf);
439     this.online = new AtomicBoolean(false);
440     this.userProvider = UserProvider.instantiate(conf);
441     FSUtils.setupShortCircuitRead(this.conf);
442 
443     // Config'ed params
444     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
445         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
446     this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
447     this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
448 
449     this.sleeper = new Sleeper(this.msgInterval, this);
450 
451     boolean isNoncesEnabled = conf.getBoolean(HConstants.HBASE_RS_NONCES_ENABLED, true);
452     this.nonceManager = isNoncesEnabled ? new ServerNonceManager(this.conf) : null;
453 
454     this.numRegionsToReport = conf.getInt(
455       "hbase.regionserver.numregionstoreport", 10);
456 
457     this.operationTimeout = conf.getInt(
458       HConstants.HBASE_RPC_SHORTOPERATION_TIMEOUT_KEY,
459       HConstants.DEFAULT_HBASE_RPC_SHORTOPERATION_TIMEOUT);
460 
461     this.abortRequested = false;
462     this.stopped = false;
463 
464     rpcServices = createRpcServices();
465     this.startcode = System.currentTimeMillis();
466     String hostName = rpcServices.isa.getHostName();
467     serverName = ServerName.valueOf(hostName, rpcServices.isa.getPort(), startcode);
468 
469     // login the zookeeper client principal (if using security)
470     ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
471       "hbase.zookeeper.client.kerberos.principal", hostName);
472     // login the server principal (if using secure Hadoop)
473     login(userProvider, hostName);
474 
475     regionServerAccounting = new RegionServerAccounting();
476     cacheConfig = new CacheConfig(conf);
477     uncaughtExceptionHandler = new UncaughtExceptionHandler() {
478       @Override
479       public void uncaughtException(Thread t, Throwable e) {
480         abort("Uncaught exception in service thread " + t.getName(), e);
481       }
482     };
483 
484     // Set 'fs.defaultFS' to match the filesystem on hbase.rootdir else
485     // underlying hadoop hdfs accessors will be going against wrong filesystem
486     // (unless all is set to defaults).
487     FSUtils.setFsDefault(this.conf, FSUtils.getRootDir(this.conf));
488     // Get fs instance used by this RS.  Do we use checksum verification in the hbase? If hbase
489     // checksum verification enabled, then automatically switch off hdfs checksum verification.
490     boolean useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, true);
491     this.fs = new HFileSystem(this.conf, useHBaseChecksum);
492     this.rootDir = FSUtils.getRootDir(this.conf);
493     this.tableDescriptors = new FSTableDescriptors(
494       this.fs, this.rootDir, !canUpdateTableDescriptor());
495 
496     service = new ExecutorService(getServerName().toShortString());
497     spanReceiverHost = SpanReceiverHost.getInstance(getConfiguration());
498 
499     // Some unit tests don't need a cluster, so no zookeeper at all
500     if (!conf.getBoolean("hbase.testing.nocluster", false)) {
501       // Open connection to zookeeper and set primary watcher
502       zooKeeper = new ZooKeeperWatcher(conf, getProcessName() + ":" +
503         rpcServices.isa.getPort(), this, canCreateBaseZNode());
504 
505       this.csm = (BaseCoordinatedStateManager) csm;
506       this.csm.initialize(this);
507       this.csm.start();
508 
509       tableLockManager = TableLockManager.createTableLockManager(
510         conf, zooKeeper, serverName);
511 
512       masterAddressTracker = new MasterAddressTracker(getZooKeeper(), this);
513       masterAddressTracker.start();
514 
515       clusterStatusTracker = new ClusterStatusTracker(zooKeeper, this);
516       clusterStatusTracker.start();
517     }
518 
519     rpcServices.start();
520     putUpWebUI();
521   }
522 
523   protected void login(UserProvider user, String host) throws IOException {
524     user.login("hbase.regionserver.keytab.file",
525       "hbase.regionserver.kerberos.principal", host);
526   }
527 
528   protected String getProcessName() {
529     return REGIONSERVER;
530   }
531 
532   protected boolean canCreateBaseZNode() {
533     return false;
534   }
535 
536   protected boolean canUpdateTableDescriptor() {
537     return false;
538   }
539 
540   protected RSRpcServices createRpcServices() throws IOException {
541     return new RSRpcServices(this);
542   }
543 
544   protected void configureInfoServer() {
545     infoServer.addServlet("rs-status", "/rs-status", RSStatusServlet.class);
546     infoServer.setAttribute(REGIONSERVER, this);
547   }
548 
549   protected Class<? extends HttpServlet> getDumpServlet() {
550     return RSDumpServlet.class;
551   }
552 
553   protected void doMetrics() {
554   }
555 
556   /**
557    * Create wrapped short-circuit connection to this server.
558    * In its own method so can intercept and mock it over in tests.
559    * @throws IOException
560    */
561   protected HConnection createShortCircuitConnection() throws IOException {
562     return ConnectionUtils.createShortCircuitHConnection(
563       HConnectionManager.getConnection(conf), serverName, rpcServices, rpcServices);
564   }
565 
566   /**
567    * Run test on configured codecs to make sure supporting libs are in place.
568    * @param c
569    * @throws IOException
570    */
571   private static void checkCodecs(final Configuration c) throws IOException {
572     // check to see if the codec list is available:
573     String [] codecs = c.getStrings("hbase.regionserver.codecs", (String[])null);
574     if (codecs == null) return;
575     for (String codec : codecs) {
576       if (!CompressionTest.testCompression(codec)) {
577         throw new IOException("Compression codec " + codec +
578           " not supported, aborting RS construction");
579       }
580     }
581   }
582 
583   public String getClusterId() {
584     return this.clusterId;
585   }
586 
587   /**
588    * All initialization needed before we go register with Master.
589    *
590    * @throws IOException
591    * @throws InterruptedException
592    */
593   private void preRegistrationInitialization(){
594     try {
595       initializeZooKeeper();
596       initializeThreads();
597     } catch (Throwable t) {
598       // Call stop if error or process will stick around for ever since server
599       // puts up non-daemon threads.
600       this.rpcServices.stop();
601       abort("Initialization of RS failed.  Hence aborting RS.", t);
602     }
603   }
604 
605   /**
606    * Bring up connection to zk ensemble and then wait until a master for this
607    * cluster and then after that, wait until cluster 'up' flag has been set.
608    * This is the order in which master does things.
609    * Finally open long-living server short-circuit connection.
610    * @throws IOException
611    * @throws InterruptedException
612    */
613   private void initializeZooKeeper() throws IOException, InterruptedException {
614     // Create the master address tracker, register with zk, and start it.  Then
615     // block until a master is available.  No point in starting up if no master
616     // running.
617     this.masterAddressTracker = new MasterAddressTracker(this.zooKeeper, this);
618     this.masterAddressTracker.start();
619     blockAndCheckIfStopped(this.masterAddressTracker);
620 
621     // Wait on cluster being up.  Master will set this flag up in zookeeper
622     // when ready.
623     blockAndCheckIfStopped(this.clusterStatusTracker);
624 
625     // Retrieve clusterId
626     // Since cluster status is now up
627     // ID should have already been set by HMaster
628     try {
629       clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
630       if (clusterId == null) {
631         this.abort("Cluster ID has not been set");
632       }
633       LOG.info("ClusterId : "+clusterId);
634     } catch (KeeperException e) {
635       this.abort("Failed to retrieve Cluster ID",e);
636     }
637 
638     shortCircuitConnection = createShortCircuitConnection();
639     metaTableLocator = new MetaTableLocator();
640 
641     // watch for snapshots and other procedures
642     try {
643       rspmHost = new RegionServerProcedureManagerHost();
644       rspmHost.loadProcedures(conf);
645       rspmHost.initialize(this);
646     } catch (KeeperException e) {
647       this.abort("Failed to reach zk cluster when creating procedure handler.", e);
648     }
649     // register watcher for recovering regions
650     this.recoveringRegionWatcher = new RecoveringRegionWatcher(this.zooKeeper, this);
651   }
652 
653   /**
654    * Utilty method to wait indefinitely on a znode availability while checking
655    * if the region server is shut down
656    * @param tracker znode tracker to use
657    * @throws IOException any IO exception, plus if the RS is stopped
658    * @throws InterruptedException
659    */
660   private void blockAndCheckIfStopped(ZooKeeperNodeTracker tracker)
661       throws IOException, InterruptedException {
662     while (tracker.blockUntilAvailable(this.msgInterval, false) == null) {
663       if (this.stopped) {
664         throw new IOException("Received the shutdown message while waiting.");
665       }
666     }
667   }
668 
669   /**
670    * @return False if cluster shutdown in progress
671    */
672   private boolean isClusterUp() {
673     return clusterStatusTracker != null && clusterStatusTracker.isClusterUp();
674   }
675 
676   private void initializeThreads() throws IOException {
677     // Cache flushing thread.
678     this.cacheFlusher = new MemStoreFlusher(conf, this);
679 
680     // Compaction thread
681     this.compactSplitThread = new CompactSplitThread(this);
682 
683     // Background thread to check for compactions; needed if region has not gotten updates
684     // in a while. It will take care of not checking too frequently on store-by-store basis.
685     this.compactionChecker = new CompactionChecker(this, this.threadWakeFrequency, this);
686     this.periodicFlusher = new PeriodicMemstoreFlusher(this.threadWakeFrequency, this);
687     // Health checker thread.
688     int sleepTime = this.conf.getInt(HConstants.HEALTH_CHORE_WAKE_FREQ,
689       HConstants.DEFAULT_THREAD_WAKE_FREQUENCY);
690     if (isHealthCheckerConfigured()) {
691       healthCheckChore = new HealthCheckChore(sleepTime, this, getConfiguration());
692     }
693 
694     this.leases = new Leases(this.threadWakeFrequency);
695 
696     // Create the thread to clean the moved regions list
697     movedRegionsCleaner = MovedRegionsCleaner.createAndStart(this);
698 
699     if (this.nonceManager != null) {
700       // Create the chore that cleans up nonces.
701       nonceManagerChore = this.nonceManager.createCleanupChore(this);
702     }
703 
704     // Setup RPC client for master communication
705     rpcClient = new RpcClient(conf, clusterId, new InetSocketAddress(
706       rpcServices.isa.getAddress(), 0));
707     this.pauseMonitor = new JvmPauseMonitor(conf);
708     pauseMonitor.start();
709 
710     int storefileRefreshPeriod = conf.getInt(StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD
711       , StorefileRefresherChore.DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD);
712     if (storefileRefreshPeriod > 0) {
713       this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod, this, this);
714     }
715   }
716 
717   /**
718    * The HRegionServer sticks in this loop until closed.
719    */
720   @Override
721   public void run() {
722     try {
723       // Do pre-registration initializations; zookeeper, lease threads, etc.
724       preRegistrationInitialization();
725     } catch (Throwable e) {
726       abort("Fatal exception during initialization", e);
727     }
728 
729     try {
730       if (!isStopped() && !isAborted()) {
731         ShutdownHook.install(conf, fs, this, Thread.currentThread());
732         // Set our ephemeral znode up in zookeeper now we have a name.
733         createMyEphemeralNode();
734         // Initialize the RegionServerCoprocessorHost now that our ephemeral
735         // node was created, in case any coprocessors want to use ZooKeeper
736         this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
737       }
738 
739       // Try and register with the Master; tell it we are here.  Break if
740       // server is stopped or the clusterup flag is down or hdfs went wacky.
741       while (keepLooping()) {
742         RegionServerStartupResponse w = reportForDuty();
743         if (w == null) {
744           LOG.warn("reportForDuty failed; sleeping and then retrying.");
745           this.sleeper.sleep();
746         } else {
747           handleReportForDutyResponse(w);
748           break;
749         }
750       }
751 
752       if (!isStopped() && isHealthy()){
753         // start the snapshot handler and other procedure handlers,
754         // since the server is ready to run
755         rspmHost.start();
756       }
757 
758       // We registered with the Master.  Go into run mode.
759       long lastMsg = System.currentTimeMillis();
760       long oldRequestCount = -1;
761       // The main run loop.
762       while (!isStopped() && isHealthy()) {
763         if (!isClusterUp()) {
764           if (isOnlineRegionsEmpty()) {
765             stop("Exiting; cluster shutdown set and not carrying any regions");
766           } else if (!this.stopping) {
767             this.stopping = true;
768             LOG.info("Closing user regions");
769             closeUserRegions(this.abortRequested);
770           } else if (this.stopping) {
771             boolean allUserRegionsOffline = areAllUserRegionsOffline();
772             if (allUserRegionsOffline) {
773               // Set stopped if no more write requests tp meta tables
774               // since last time we went around the loop.  Any open
775               // meta regions will be closed on our way out.
776               if (oldRequestCount == getWriteRequestCount()) {
777                 stop("Stopped; only catalog regions remaining online");
778                 break;
779               }
780               oldRequestCount = getWriteRequestCount();
781             } else {
782               // Make sure all regions have been closed -- some regions may
783               // have not got it because we were splitting at the time of
784               // the call to closeUserRegions.
785               closeUserRegions(this.abortRequested);
786             }
787             LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
788           }
789         }
790         long now = System.currentTimeMillis();
791         if ((now - lastMsg) >= msgInterval) {
792           tryRegionServerReport(lastMsg, now);
793           lastMsg = System.currentTimeMillis();
794           doMetrics();
795         }
796         if (!isStopped() && !isAborted()) {
797           this.sleeper.sleep();
798         }
799       } // for
800     } catch (Throwable t) {
801       if (!rpcServices.checkOOME(t)) {
802         String prefix = t instanceof YouAreDeadException? "": "Unhandled: ";
803         abort(prefix + t.getMessage(), t);
804       }
805     }
806     // Run shutdown.
807     if (mxBean != null) {
808       MBeanUtil.unregisterMBean(mxBean);
809       mxBean = null;
810     }
811     if (this.leases != null) this.leases.closeAfterLeasesExpire();
812     if (this.splitLogWorker != null) {
813       splitLogWorker.stop();
814     }
815     if (this.infoServer != null) {
816       LOG.info("Stopping infoServer");
817       try {
818         this.infoServer.stop();
819       } catch (Exception e) {
820         LOG.error("Failed to stop infoServer", e);
821       }
822     }
823     // Send cache a shutdown.
824     if (cacheConfig.isBlockCacheEnabled()) {
825       cacheConfig.getBlockCache().shutdown();
826     }
827 
828     if (movedRegionsCleaner != null) {
829       movedRegionsCleaner.stop("Region Server stopping");
830     }
831 
832     // Send interrupts to wake up threads if sleeping so they notice shutdown.
833     // TODO: Should we check they are alive? If OOME could have exited already
834     if(this.hMemManager != null) this.hMemManager.stop();
835     if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
836     if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
837     if (this.compactionChecker != null)
838       this.compactionChecker.interrupt();
839     if (this.healthCheckChore != null) {
840       this.healthCheckChore.interrupt();
841     }
842     if (this.nonceManagerChore != null) {
843       this.nonceManagerChore.interrupt();
844     }
845     if (this.storefileRefresher != null) {
846       this.storefileRefresher.interrupt();
847     }
848 
849     // Stop the snapshot and other procedure handlers, forcefully killing all running tasks
850     if (rspmHost != null) {
851       rspmHost.stop(this.abortRequested || this.killed);
852     }
853 
854     if (this.killed) {
855       // Just skip out w/o closing regions.  Used when testing.
856     } else if (abortRequested) {
857       if (this.fsOk) {
858         closeUserRegions(abortRequested); // Don't leave any open file handles
859       }
860       LOG.info("aborting server " + this.serverName);
861     } else {
862       closeUserRegions(abortRequested);
863       LOG.info("stopping server " + this.serverName);
864     }
865 
866     // so callers waiting for meta without timeout can stop
867     metaTableLocator.stop();
868     if (this.shortCircuitConnection != null && !shortCircuitConnection.isClosed()) {
869       try {
870         this.shortCircuitConnection.close();
871       } catch (IOException e) {
872         // Although the {@link Closeable} interface throws an {@link
873         // IOException}, in reality, the implementation would never do that.
874         LOG.error("Attempt to close server's short circuit HConnection failed.", e);
875       }
876     }
877 
878     // Closing the compactSplit thread before closing meta regions
879     if (!this.killed && containsMetaTableRegions()) {
880       if (!abortRequested || this.fsOk) {
881         if (this.compactSplitThread != null) {
882           this.compactSplitThread.join();
883           this.compactSplitThread = null;
884         }
885         closeMetaTableRegions(abortRequested);
886       }
887     }
888 
889     if (!this.killed && this.fsOk) {
890       waitOnAllRegionsToClose(abortRequested);
891       LOG.info("stopping server " + this.serverName +
892         "; all regions closed.");
893     }
894 
895     //fsOk flag may be changed when closing regions throws exception.
896     if (this.fsOk) {
897       closeWAL(!abortRequested);
898     }
899 
900     // Make sure the proxy is down.
901     if (this.rssStub != null) {
902       this.rssStub = null;
903     }
904     if (this.rpcClient != null) {
905       this.rpcClient.stop();
906     }
907     if (this.leases != null) {
908       this.leases.close();
909     }
910     if (this.pauseMonitor != null) {
911       this.pauseMonitor.stop();
912     }
913 
914     if (!killed) {
915       stopServiceThreads();
916     }
917 
918     if (this.rpcServices != null) {
919       this.rpcServices.stop();
920     }
921 
922     try {
923       deleteMyEphemeralNode();
924     } catch (KeeperException e) {
925       LOG.warn("Failed deleting my ephemeral node", e);
926     }
927     // We may have failed to delete the znode at the previous step, but
928     //  we delete the file anyway: a second attempt to delete the znode is likely to fail again.
929     ZNodeClearer.deleteMyEphemeralNodeOnDisk();
930 
931     if (this.zooKeeper != null) {
932       this.zooKeeper.close();
933     }
934     LOG.info("stopping server " + this.serverName +
935       "; zookeeper connection closed.");
936 
937     LOG.info(Thread.currentThread().getName() + " exiting");
938   }
939 
940   private boolean containsMetaTableRegions() {
941     return onlineRegions.containsKey(HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
942   }
943 
944   private boolean areAllUserRegionsOffline() {
945     if (getNumberOfOnlineRegions() > 2) return false;
946     boolean allUserRegionsOffline = true;
947     for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
948       if (!e.getValue().getRegionInfo().isMetaTable()) {
949         allUserRegionsOffline = false;
950         break;
951       }
952     }
953     return allUserRegionsOffline;
954   }
955 
956   /**
957    * @return Current write count for all online regions.
958    */
959   private long getWriteRequestCount() {
960     int writeCount = 0;
961     for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
962       writeCount += e.getValue().getWriteRequestsCount();
963     }
964     return writeCount;
965   }
966 
967   @VisibleForTesting
968   protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
969   throws IOException {
970     RegionServerStatusService.BlockingInterface rss = rssStub;
971     if (rss == null) {
972       // the current server could be stopping.
973       return;
974     }
975     ClusterStatusProtos.ServerLoad sl = buildServerLoad(reportStartTime, reportEndTime);
976     try {
977       RegionServerReportRequest.Builder request = RegionServerReportRequest.newBuilder();
978       ServerName sn = ServerName.parseVersionedServerName(
979         this.serverName.getVersionedBytes());
980       request.setServer(ProtobufUtil.toServerName(sn));
981       request.setLoad(sl);
982       rss.regionServerReport(null, request.build());
983     } catch (ServiceException se) {
984       IOException ioe = ProtobufUtil.getRemoteException(se);
985       if (ioe instanceof YouAreDeadException) {
986         // This will be caught and handled as a fatal error in run()
987         throw ioe;
988       }
989       if (rssStub == rss) {
990         rssStub = null;
991       }
992       // Couldn't connect to the master, get location from zk and reconnect
993       // Method blocks until new master is found or we are stopped
994       createRegionServerStatusStub();
995     }
996   }
997 
998   ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime) {
999     // We're getting the MetricsRegionServerWrapper here because the wrapper computes requests
1000     // per second, and other metrics  As long as metrics are part of ServerLoad it's best to use
1001     // the wrapper to compute those numbers in one place.
1002     // In the long term most of these should be moved off of ServerLoad and the heart beat.
1003     // Instead they should be stored in an HBase table so that external visibility into HBase is
1004     // improved; Additionally the load balancer will be able to take advantage of a more complete
1005     // history.
1006     MetricsRegionServerWrapper regionServerWrapper = this.metricsRegionServer.getRegionServerWrapper();
1007     Collection<HRegion> regions = getOnlineRegionsLocalContext();
1008     MemoryUsage memory =
1009       ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
1010 
1011     ClusterStatusProtos.ServerLoad.Builder serverLoad =
1012       ClusterStatusProtos.ServerLoad.newBuilder();
1013     serverLoad.setNumberOfRequests((int) regionServerWrapper.getRequestsPerSecond());
1014     serverLoad.setTotalNumberOfRequests((int) regionServerWrapper.getTotalRequestCount());
1015     serverLoad.setUsedHeapMB((int)(memory.getUsed() / 1024 / 1024));
1016     serverLoad.setMaxHeapMB((int) (memory.getMax() / 1024 / 1024));
1017     Set<String> coprocessors = this.hlog.getCoprocessorHost().getCoprocessors();
1018     for (String coprocessor : coprocessors) {
1019       serverLoad.addCoprocessors(
1020         Coprocessor.newBuilder().setName(coprocessor).build());
1021     }
1022     RegionLoad.Builder regionLoadBldr = RegionLoad.newBuilder();
1023     RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
1024     for (HRegion region : regions) {
1025       serverLoad.addRegionLoads(createRegionLoad(region, regionLoadBldr, regionSpecifier));
1026     }
1027     serverLoad.setReportStartTime(reportStartTime);
1028     serverLoad.setReportEndTime(reportEndTime);
1029     if (this.infoServer != null) {
1030       serverLoad.setInfoServerPort(this.infoServer.getPort());
1031     } else {
1032       serverLoad.setInfoServerPort(-1);
1033     }
1034     return serverLoad.build();
1035   }
1036 
1037   String getOnlineRegionsAsPrintableString() {
1038     StringBuilder sb = new StringBuilder();
1039     for (HRegion r: this.onlineRegions.values()) {
1040       if (sb.length() > 0) sb.append(", ");
1041       sb.append(r.getRegionInfo().getEncodedName());
1042     }
1043     return sb.toString();
1044   }
1045 
1046   /**
1047    * Wait on regions close.
1048    */
1049   private void waitOnAllRegionsToClose(final boolean abort) {
1050     // Wait till all regions are closed before going out.
1051     int lastCount = -1;
1052     long previousLogTime = 0;
1053     Set<String> closedRegions = new HashSet<String>();
1054     boolean interrupted = false;
1055     try {
1056       while (!isOnlineRegionsEmpty()) {
1057         int count = getNumberOfOnlineRegions();
1058         // Only print a message if the count of regions has changed.
1059         if (count != lastCount) {
1060           // Log every second at most
1061           if (System.currentTimeMillis() > (previousLogTime + 1000)) {
1062             previousLogTime = System.currentTimeMillis();
1063             lastCount = count;
1064             LOG.info("Waiting on " + count + " regions to close");
1065             // Only print out regions still closing if a small number else will
1066             // swamp the log.
1067             if (count < 10 && LOG.isDebugEnabled()) {
1068               LOG.debug(this.onlineRegions);
1069             }
1070           }
1071         }
1072         // Ensure all user regions have been sent a close. Use this to
1073         // protect against the case where an open comes in after we start the
1074         // iterator of onlineRegions to close all user regions.
1075         for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
1076           HRegionInfo hri = e.getValue().getRegionInfo();
1077           if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes())
1078               && !closedRegions.contains(hri.getEncodedName())) {
1079             closedRegions.add(hri.getEncodedName());
1080             // Don't update zk with this close transition; pass false.
1081             closeRegionIgnoreErrors(hri, abort);
1082               }
1083         }
1084         // No regions in RIT, we could stop waiting now.
1085         if (this.regionsInTransitionInRS.isEmpty()) {
1086           if (!isOnlineRegionsEmpty()) {
1087             LOG.info("We were exiting though online regions are not empty," +
1088                 " because some regions failed closing");
1089           }
1090           break;
1091         }
1092         try {
1093           Thread.sleep(200);
1094         } catch (InterruptedException e) {
1095           interrupted = true;
1096           LOG.warn("Interrupted while sleeping");
1097         }
1098       }
1099     } finally {
1100       if (interrupted) {
1101         Thread.currentThread().interrupt();
1102       }
1103     }
1104   }
1105 
1106   private void closeWAL(final boolean delete) {
1107     if (this.hlogForMeta != null) {
1108       // All hlogs (meta and non-meta) are in the same directory. Don't call
1109       // closeAndDelete here since that would delete all hlogs not just the
1110       // meta ones. We will just 'close' the hlog for meta here, and leave
1111       // the directory cleanup to the follow-on closeAndDelete call.
1112       try {
1113         this.hlogForMeta.close();
1114       } catch (Throwable e) {
1115         e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
1116         LOG.error("Metalog close and delete failed", e);
1117       }
1118     }
1119     if (this.hlog != null) {
1120       try {
1121         if (delete) {
1122           hlog.closeAndDelete();
1123         } else {
1124           hlog.close();
1125         }
1126       } catch (Throwable e) {
1127         e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
1128         LOG.error("Close and delete failed", e);
1129       }
1130     }
1131   }
1132 
1133   /*
1134    * Run init. Sets up hlog and starts up all server threads.
1135    *
1136    * @param c Extra configuration.
1137    */
1138   protected void handleReportForDutyResponse(final RegionServerStartupResponse c)
1139   throws IOException {
1140     try {
1141       for (NameStringPair e : c.getMapEntriesList()) {
1142         String key = e.getName();
1143         // The hostname the master sees us as.
1144         if (key.equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
1145           String hostnameFromMasterPOV = e.getValue();
1146           this.serverName = ServerName.valueOf(hostnameFromMasterPOV,
1147             rpcServices.isa.getPort(), this.startcode);
1148           if (!hostnameFromMasterPOV.equals(rpcServices.isa.getHostName())) {
1149             LOG.info("Master passed us a different hostname to use; was=" +
1150               rpcServices.isa.getHostName() + ", but now=" + hostnameFromMasterPOV);
1151           }
1152           continue;
1153         }
1154         String value = e.getValue();
1155         if (LOG.isDebugEnabled()) {
1156           LOG.info("Config from master: " + key + "=" + value);
1157         }
1158         this.conf.set(key, value);
1159       }
1160 
1161       // hack! Maps DFSClient => RegionServer for logs.  HDFS made this
1162       // config param for task trackers, but we can piggyback off of it.
1163       if (this.conf.get("mapreduce.task.attempt.id") == null) {
1164         this.conf.set("mapreduce.task.attempt.id", "hb_rs_" +
1165           this.serverName.toString());
1166       }
1167 
1168       // Save it in a file, this will allow to see if we crash
1169       ZNodeClearer.writeMyEphemeralNodeOnDisk(getMyEphemeralNodePath());
1170 
1171       this.hlog = setupWALAndReplication();
1172       // Init in here rather than in constructor after thread name has been set
1173       this.metricsRegionServer = new MetricsRegionServer(new MetricsRegionServerWrapperImpl(this));
1174 
1175       startServiceThreads();
1176       startHeapMemoryManager();
1177       LOG.info("Serving as " + this.serverName +
1178         ", RpcServer on " + rpcServices.isa +
1179         ", sessionid=0x" +
1180         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
1181       synchronized (online) {
1182         online.set(true);
1183         online.notifyAll();
1184       }
1185     } catch (Throwable e) {
1186       stop("Failed initialization");
1187       throw convertThrowableToIOE(cleanup(e, "Failed init"),
1188           "Region server startup failed");
1189     } finally {
1190       sleeper.skipSleepCycle();
1191     }
1192   }
1193 
1194   private void startHeapMemoryManager() {
1195     this.hMemManager = HeapMemoryManager.create(this.conf, this.cacheFlusher, this);
1196     if (this.hMemManager != null) {
1197       this.hMemManager.start();
1198     }
1199   }
1200 
1201   private void createMyEphemeralNode() throws KeeperException, IOException {
1202     RegionServerInfo.Builder rsInfo = RegionServerInfo.newBuilder();
1203     rsInfo.setInfoPort(infoServer != null ? infoServer.getPort() : -1);
1204     byte[] data = ProtobufUtil.prependPBMagic(rsInfo.build().toByteArray());
1205     ZKUtil.createEphemeralNodeAndWatch(this.zooKeeper,
1206       getMyEphemeralNodePath(), data);
1207   }
1208 
1209   private void deleteMyEphemeralNode() throws KeeperException {
1210     ZKUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath());
1211   }
1212 
1213   @Override
1214   public RegionServerAccounting getRegionServerAccounting() {
1215     return regionServerAccounting;
1216   }
1217 
1218   @Override
1219   public TableLockManager getTableLockManager() {
1220     return tableLockManager;
1221   }
1222 
1223   /*
1224    * @param r Region to get RegionLoad for.
1225    * @param regionLoadBldr the RegionLoad.Builder, can be null
1226    * @param regionSpecifier the RegionSpecifier.Builder, can be null
1227    * @return RegionLoad instance.
1228    *
1229    * @throws IOException
1230    */
1231   private RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr,
1232       RegionSpecifier.Builder regionSpecifier) {
1233     byte[] name = r.getRegionName();
1234     int stores = 0;
1235     int storefiles = 0;
1236     int storeUncompressedSizeMB = 0;
1237     int storefileSizeMB = 0;
1238     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
1239     int storefileIndexSizeMB = 0;
1240     int rootIndexSizeKB = 0;
1241     int totalStaticIndexSizeKB = 0;
1242     int totalStaticBloomSizeKB = 0;
1243     long totalCompactingKVs = 0;
1244     long currentCompactedKVs = 0;
1245     synchronized (r.stores) {
1246       stores += r.stores.size();
1247       for (Store store : r.stores.values()) {
1248         storefiles += store.getStorefilesCount();
1249         storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed()
1250             / 1024 / 1024);
1251         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
1252         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
1253         CompactionProgress progress = store.getCompactionProgress();
1254         if (progress != null) {
1255           totalCompactingKVs += progress.totalCompactingKVs;
1256           currentCompactedKVs += progress.currentCompactedKVs;
1257         }
1258 
1259         rootIndexSizeKB +=
1260             (int) (store.getStorefilesIndexSize() / 1024);
1261 
1262         totalStaticIndexSizeKB +=
1263           (int) (store.getTotalStaticIndexSize() / 1024);
1264 
1265         totalStaticBloomSizeKB +=
1266           (int) (store.getTotalStaticBloomSize() / 1024);
1267       }
1268     }
1269     if (regionLoadBldr == null) {
1270       regionLoadBldr = RegionLoad.newBuilder();
1271     }
1272     if (regionSpecifier == null) {
1273       regionSpecifier = RegionSpecifier.newBuilder();
1274     }
1275     regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
1276     regionSpecifier.setValue(ByteStringer.wrap(name));
1277     regionLoadBldr.setRegionSpecifier(regionSpecifier.build())
1278       .setStores(stores)
1279       .setStorefiles(storefiles)
1280       .setStoreUncompressedSizeMB(storeUncompressedSizeMB)
1281       .setStorefileSizeMB(storefileSizeMB)
1282       .setMemstoreSizeMB(memstoreSizeMB)
1283       .setStorefileIndexSizeMB(storefileIndexSizeMB)
1284       .setRootIndexSizeKB(rootIndexSizeKB)
1285       .setTotalStaticIndexSizeKB(totalStaticIndexSizeKB)
1286       .setTotalStaticBloomSizeKB(totalStaticBloomSizeKB)
1287       .setReadRequestsCount(r.readRequestsCount.get())
1288       .setWriteRequestsCount(r.writeRequestsCount.get())
1289       .setTotalCompactingKVs(totalCompactingKVs)
1290       .setCurrentCompactedKVs(currentCompactedKVs)
1291       .setCompleteSequenceId(r.lastFlushSeqId);
1292 
1293     return regionLoadBldr.build();
1294   }
1295 
1296   /**
1297    * @param encodedRegionName
1298    * @return An instance of RegionLoad.
1299    */
1300   public RegionLoad createRegionLoad(final String encodedRegionName) {
1301     HRegion r = null;
1302     r = this.onlineRegions.get(encodedRegionName);
1303     return r != null ? createRegionLoad(r, null, null) : null;
1304   }
1305 
1306   /*
1307    * Inner class that runs on a long period checking if regions need compaction.
1308    */
1309   private static class CompactionChecker extends Chore {
1310     private final HRegionServer instance;
1311     private final int majorCompactPriority;
1312     private final static int DEFAULT_PRIORITY = Integer.MAX_VALUE;
1313     private long iteration = 0;
1314 
1315     CompactionChecker(final HRegionServer h, final int sleepTime,
1316         final Stoppable stopper) {
1317       super("CompactionChecker", sleepTime, h);
1318       this.instance = h;
1319       LOG.info(this.getName() + " runs every " + StringUtils.formatTime(sleepTime));
1320 
1321       /* MajorCompactPriority is configurable.
1322        * If not set, the compaction will use default priority.
1323        */
1324       this.majorCompactPriority = this.instance.conf.
1325         getInt("hbase.regionserver.compactionChecker.majorCompactPriority",
1326         DEFAULT_PRIORITY);
1327     }
1328 
1329     @Override
1330     protected void chore() {
1331       for (HRegion r : this.instance.onlineRegions.values()) {
1332         if (r == null)
1333           continue;
1334         for (Store s : r.getStores().values()) {
1335           try {
1336             long multiplier = s.getCompactionCheckMultiplier();
1337             assert multiplier > 0;
1338             if (iteration % multiplier != 0) continue;
1339             if (s.needsCompaction()) {
1340               // Queue a compaction. Will recognize if major is needed.
1341               this.instance.compactSplitThread.requestSystemCompaction(r, s, getName()
1342                   + " requests compaction");
1343             } else if (s.isMajorCompaction()) {
1344               if (majorCompactPriority == DEFAULT_PRIORITY
1345                   || majorCompactPriority > r.getCompactPriority()) {
1346                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1347                     + " requests major compaction; use default priority", null);
1348               } else {
1349                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1350                     + " requests major compaction; use configured priority",
1351                   this.majorCompactPriority, null);
1352               }
1353             }
1354           } catch (IOException e) {
1355             LOG.warn("Failed major compaction check on " + r, e);
1356           }
1357         }
1358       }
1359       iteration = (iteration == Long.MAX_VALUE) ? 0 : (iteration + 1);
1360     }
1361   }
1362 
1363   class PeriodicMemstoreFlusher extends Chore {
1364     final HRegionServer server;
1365     final static int RANGE_OF_DELAY = 20000; //millisec
1366     final static int MIN_DELAY_TIME = 3000; //millisec
1367     public PeriodicMemstoreFlusher(int cacheFlushInterval, final HRegionServer server) {
1368       super(server.getServerName() + "-MemstoreFlusherChore", cacheFlushInterval, server);
1369       this.server = server;
1370     }
1371 
1372     @Override
1373     protected void chore() {
1374       for (HRegion r : this.server.onlineRegions.values()) {
1375         if (r == null)
1376           continue;
1377         if (r.shouldFlush()) {
1378           FlushRequester requester = server.getFlushRequester();
1379           if (requester != null) {
1380             long randomDelay = RandomUtils.nextInt(RANGE_OF_DELAY) + MIN_DELAY_TIME;
1381             LOG.info(getName() + " requesting flush for region " + r.getRegionNameAsString() +
1382                 " after a delay of " + randomDelay);
1383             //Throttle the flushes by putting a delay. If we don't throttle, and there
1384             //is a balanced write-load on the regions in a table, we might end up
1385             //overwhelming the filesystem with too many flushes at once.
1386             requester.requestDelayedFlush(r, randomDelay);
1387           }
1388         }
1389       }
1390     }
1391   }
1392 
1393   /**
1394    * Report the status of the server. A server is online once all the startup is
1395    * completed (setting up filesystem, starting service threads, etc.). This
1396    * method is designed mostly to be useful in tests.
1397    *
1398    * @return true if online, false if not.
1399    */
1400   public boolean isOnline() {
1401     return online.get();
1402   }
1403 
1404   /**
1405    * Setup WAL log and replication if enabled.
1406    * Replication setup is done in here because it wants to be hooked up to WAL.
1407    * @return A WAL instance.
1408    * @throws IOException
1409    */
1410   private HLog setupWALAndReplication() throws IOException {
1411     final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1412     final String logName
1413       = HLogUtil.getHLogDirectoryName(this.serverName.toString());
1414 
1415     Path logdir = new Path(rootDir, logName);
1416     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1417     if (this.fs.exists(logdir)) {
1418       throw new RegionServerRunningException("Region server has already " +
1419         "created directory at " + this.serverName.toString());
1420     }
1421 
1422     // Instantiate replication manager if replication enabled.  Pass it the
1423     // log directories.
1424     createNewReplicationInstance(conf, this, this.fs, logdir, oldLogDir);
1425 
1426     return instantiateHLog(rootDir, logName);
1427   }
1428 
1429   private HLog getMetaWAL() throws IOException {
1430     if (this.hlogForMeta != null) return this.hlogForMeta;
1431     final String logName = HLogUtil.getHLogDirectoryName(this.serverName.toString());
1432     Path logdir = new Path(rootDir, logName);
1433     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1434     this.hlogForMeta = HLogFactory.createMetaHLog(this.fs.getBackingFs(), rootDir, logName,
1435       this.conf, getMetaWALActionListeners(), this.serverName.toString());
1436     return this.hlogForMeta;
1437   }
1438 
1439   /**
1440    * Called by {@link #setupWALAndReplication()} creating WAL instance.
1441    * @param rootdir
1442    * @param logName
1443    * @return WAL instance.
1444    * @throws IOException
1445    */
1446   protected HLog instantiateHLog(Path rootdir, String logName) throws IOException {
1447     return HLogFactory.createHLog(this.fs.getBackingFs(), rootdir, logName, this.conf,
1448       getWALActionListeners(), this.serverName.toString());
1449   }
1450 
1451   /**
1452    * Called by {@link #instantiateHLog(Path, String)} setting up WAL instance.
1453    * Add any {@link WALActionsListener}s you want inserted before WAL startup.
1454    * @return List of WALActionsListener that will be passed in to
1455    * {@link org.apache.hadoop.hbase.regionserver.wal.FSHLog} on construction.
1456    */
1457   protected List<WALActionsListener> getWALActionListeners() {
1458     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1459     // Log roller.
1460     this.hlogRoller = new LogRoller(this, this);
1461     listeners.add(this.hlogRoller);
1462     if (this.replicationSourceHandler != null &&
1463         this.replicationSourceHandler.getWALActionsListener() != null) {
1464       // Replication handler is an implementation of WALActionsListener.
1465       listeners.add(this.replicationSourceHandler.getWALActionsListener());
1466     }
1467     return listeners;
1468   }
1469 
1470   protected List<WALActionsListener> getMetaWALActionListeners() {
1471     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1472     // Using a tmp log roller to ensure metaLogRoller is alive once it is not
1473     // null
1474     MetaLogRoller tmpLogRoller = new MetaLogRoller(this, this);
1475     String n = Thread.currentThread().getName();
1476     Threads.setDaemonThreadRunning(tmpLogRoller.getThread(),
1477         n + "-MetaLogRoller", uncaughtExceptionHandler);
1478     this.metaHLogRoller = tmpLogRoller;
1479     tmpLogRoller = null;
1480     listeners.add(this.metaHLogRoller);
1481     return listeners;
1482   }
1483 
1484   protected LogRoller getLogRoller() {
1485     return hlogRoller;
1486   }
1487 
1488   public MetricsRegionServer getRegionServerMetrics() {
1489     return this.metricsRegionServer;
1490   }
1491 
1492   /**
1493    * @return Master address tracker instance.
1494    */
1495   public MasterAddressTracker getMasterAddressTracker() {
1496     return this.masterAddressTracker;
1497   }
1498 
1499   /*
1500    * Start maintenance Threads, Server, Worker and lease checker threads.
1501    * Install an UncaughtExceptionHandler that calls abort of RegionServer if we
1502    * get an unhandled exception. We cannot set the handler on all threads.
1503    * Server's internal Listener thread is off limits. For Server, if an OOME, it
1504    * waits a while then retries. Meantime, a flush or a compaction that tries to
1505    * run should trigger same critical condition and the shutdown will run. On
1506    * its way out, this server will shut down Server. Leases are sort of
1507    * inbetween. It has an internal thread that while it inherits from Chore, it
1508    * keeps its own internal stop mechanism so needs to be stopped by this
1509    * hosting server. Worker logs the exception and exits.
1510    */
1511   private void startServiceThreads() throws IOException {
1512     // Start executor services
1513     this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
1514       conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1515     this.service.startExecutorService(ExecutorType.RS_OPEN_META,
1516       conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1517     this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1518       conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1519     this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
1520       conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1521     if (conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false)) {
1522       this.service.startExecutorService(ExecutorType.RS_PARALLEL_SEEK,
1523         conf.getInt("hbase.storescanner.parallel.seek.threads", 10));
1524     }
1525     this.service.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS,
1526       conf.getInt("hbase.regionserver.wal.max.splitters", SplitLogWorker.DEFAULT_MAX_SPLITTERS));
1527 
1528     Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), getName() + ".logRoller",
1529         uncaughtExceptionHandler);
1530     this.cacheFlusher.start(uncaughtExceptionHandler);
1531     Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), getName() +
1532       ".compactionChecker", uncaughtExceptionHandler);
1533     Threads.setDaemonThreadRunning(this.periodicFlusher.getThread(), getName() +
1534         ".periodicFlusher", uncaughtExceptionHandler);
1535     if (this.healthCheckChore != null) {
1536       Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), getName() + ".healthChecker",
1537             uncaughtExceptionHandler);
1538     }
1539     if (this.nonceManagerChore != null) {
1540       Threads.setDaemonThreadRunning(this.nonceManagerChore.getThread(), getName() + ".nonceCleaner",
1541             uncaughtExceptionHandler);
1542     }
1543     if (this.storefileRefresher != null) {
1544       Threads.setDaemonThreadRunning(this.storefileRefresher.getThread(), getName() + ".storefileRefresher",
1545             uncaughtExceptionHandler);
1546     }
1547 
1548     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
1549     // an unhandled exception, it will just exit.
1550     this.leases.setName(getName() + ".leaseChecker");
1551     this.leases.start();
1552 
1553     if (this.replicationSourceHandler == this.replicationSinkHandler &&
1554         this.replicationSourceHandler != null) {
1555       this.replicationSourceHandler.startReplicationService();
1556     } else {
1557       if (this.replicationSourceHandler != null) {
1558         this.replicationSourceHandler.startReplicationService();
1559       }
1560       if (this.replicationSinkHandler != null) {
1561         this.replicationSinkHandler.startReplicationService();
1562       }
1563     }
1564 
1565     // Start Server.  This service is like leases in that it internally runs
1566     // a thread.
1567     rpcServices.rpcServer.start();
1568 
1569     // Create the log splitting worker and start it
1570     // set a smaller retries to fast fail otherwise splitlogworker could be blocked for
1571     // quite a while inside HConnection layer. The worker won't be available for other
1572     // tasks even after current task is preempted after a split task times out.
1573     Configuration sinkConf = HBaseConfiguration.create(conf);
1574     sinkConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
1575       conf.getInt("hbase.log.replay.retries.number", 8)); // 8 retries take about 23 seconds
1576     sinkConf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
1577       conf.getInt("hbase.log.replay.rpc.timeout", 30000)); // default 30 seconds
1578     sinkConf.setInt("hbase.client.serverside.retries.multiplier", 1);
1579     this.splitLogWorker = new SplitLogWorker(this.zooKeeper, sinkConf, this, this);
1580     splitLogWorker.start();
1581   }
1582 
1583   /**
1584    * Puts up the webui.
1585    * @return Returns final port -- maybe different from what we started with.
1586    * @throws IOException
1587    */
1588   private int putUpWebUI() throws IOException {
1589     int port = this.conf.getInt(HConstants.REGIONSERVER_INFO_PORT,
1590       HConstants.DEFAULT_REGIONSERVER_INFOPORT);
1591     // -1 is for disabling info server
1592     if (port < 0) return port;
1593     String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
1594     // check if auto port bind enabled
1595     boolean auto = this.conf.getBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO,
1596         false);
1597     while (true) {
1598       try {
1599         this.infoServer = new InfoServer(getProcessName(), addr, port, false, this.conf);
1600         infoServer.addServlet("dump", "/dump", getDumpServlet());
1601         configureInfoServer();
1602         this.infoServer.start();
1603         break;
1604       } catch (BindException e) {
1605         if (!auto) {
1606           // auto bind disabled throw BindException
1607           LOG.error("Failed binding http info server to port: " + port);
1608           throw e;
1609         }
1610         // auto bind enabled, try to use another port
1611         LOG.info("Failed binding http info server to port: " + port);
1612         port++;
1613       }
1614     }
1615     port = this.infoServer.getPort();
1616     conf.setInt(HConstants.REGIONSERVER_INFO_PORT, port);
1617     int masterInfoPort = conf.getInt(HConstants.MASTER_INFO_PORT,
1618       HConstants.DEFAULT_MASTER_INFOPORT);
1619     conf.setInt("hbase.master.info.port.orig", masterInfoPort);
1620     conf.setInt(HConstants.MASTER_INFO_PORT, port);
1621     return port;
1622   }
1623 
1624   /*
1625    * Verify that server is healthy
1626    */
1627   private boolean isHealthy() {
1628     if (!fsOk) {
1629       // File system problem
1630       return false;
1631     }
1632     // Verify that all threads are alive
1633     if (!(leases.isAlive()
1634         && cacheFlusher.isAlive() && hlogRoller.isAlive()
1635         && this.compactionChecker.isAlive()
1636         && this.periodicFlusher.isAlive())) {
1637       stop("One or more threads are no longer alive -- stop");
1638       return false;
1639     }
1640     if (metaHLogRoller != null && !metaHLogRoller.isAlive()) {
1641       stop("Meta HLog roller thread is no longer alive -- stop");
1642       return false;
1643     }
1644     return true;
1645   }
1646 
1647   public HLog getWAL() {
1648     try {
1649       return getWAL(null);
1650     } catch (IOException e) {
1651       LOG.warn("getWAL threw exception " + e);
1652       return null;
1653     }
1654   }
1655 
1656   @Override
1657   public HLog getWAL(HRegionInfo regionInfo) throws IOException {
1658     //TODO: at some point this should delegate to the HLogFactory
1659     //currently, we don't care about the region as much as we care about the
1660     //table.. (hence checking the tablename below)
1661     //_ROOT_ and hbase:meta regions have separate WAL.
1662     if (regionInfo != null && regionInfo.isMetaTable()) {
1663       return getMetaWAL();
1664     }
1665     return this.hlog;
1666   }
1667 
1668   @Override
1669   public HConnection getShortCircuitConnection() {
1670     return this.shortCircuitConnection;
1671   }
1672 
1673   @Override
1674   public MetaTableLocator getMetaTableLocator() {
1675     return this.metaTableLocator;
1676   }
1677 
1678   @Override
1679   public void stop(final String msg) {
1680     if (!this.stopped) {
1681       try {
1682         if (this.rsHost != null) {
1683           this.rsHost.preStop(msg);
1684         }
1685         this.stopped = true;
1686         LOG.info("STOPPED: " + msg);
1687         // Wakes run() if it is sleeping
1688         sleeper.skipSleepCycle();
1689       } catch (IOException exp) {
1690         LOG.warn("The region server did not stop", exp);
1691       }
1692     }
1693   }
1694 
1695   public void waitForServerOnline(){
1696     while (!isOnline() && !isStopped()){
1697        sleeper.sleep();
1698     }
1699   }
1700 
1701   @Override
1702   public void postOpenDeployTasks(final HRegion r)
1703   throws KeeperException, IOException {
1704     rpcServices.checkOpen();
1705     LOG.info("Post open deploy tasks for " + r.getRegionNameAsString());
1706     // Do checks to see if we need to compact (references or too many files)
1707     for (Store s : r.getStores().values()) {
1708       if (s.hasReferences() || s.needsCompaction()) {
1709        this.compactSplitThread.requestSystemCompaction(r, s, "Opening Region");
1710       }
1711     }
1712     long openSeqNum = r.getOpenSeqNum();
1713     if (openSeqNum == HConstants.NO_SEQNUM) {
1714       // If we opened a region, we should have read some sequence number from it.
1715       LOG.error("No sequence number found when opening " + r.getRegionNameAsString());
1716       openSeqNum = 0;
1717     }
1718 
1719     // Update flushed sequence id of a recovering region in ZK
1720     updateRecoveringRegionLastFlushedSequenceId(r);
1721 
1722     if (r.getRegionInfo().isMetaRegion()) {
1723       MetaTableLocator.setMetaLocation(getZooKeeper(), serverName);
1724     }
1725 
1726     // Notify master
1727     if (!reportRegionStateTransition(
1728         TransitionCode.OPENED, openSeqNum, r.getRegionInfo())) {
1729       throw new IOException("Failed to report opened region to master: "
1730         + r.getRegionNameAsString());
1731     }
1732 
1733     LOG.debug("Finished post open deploy task for " + r.getRegionNameAsString());
1734   }
1735 
1736   @Override
1737   public boolean reportRegionStateTransition(TransitionCode code, HRegionInfo... hris) {
1738     return reportRegionStateTransition(code, HConstants.NO_SEQNUM, hris);
1739   }
1740 
1741   @Override
1742   public boolean reportRegionStateTransition(
1743       TransitionCode code, long openSeqNum, HRegionInfo... hris) {
1744     if (TEST_SKIP_REPORTING_TRANSITION) {
1745       // This is for testing only in case there is no master
1746       // to handle the region transition report at all.
1747       if (code == TransitionCode.OPENED) {
1748         Preconditions.checkArgument(hris != null && hris.length == 1);
1749         if (!hris[0].isMetaRegion()) {
1750           try {
1751             MetaTableAccessor.updateRegionLocation(shortCircuitConnection,
1752               hris[0], serverName, openSeqNum);
1753           } catch (IOException e) {
1754             LOG.info("Failed to update meta", e);
1755             return false;
1756           }
1757         }
1758       }
1759       return true;
1760     }
1761 
1762     ReportRegionStateTransitionRequest.Builder builder =
1763       ReportRegionStateTransitionRequest.newBuilder();
1764     builder.setServer(ProtobufUtil.toServerName(serverName));
1765     RegionStateTransition.Builder transition = builder.addTransitionBuilder();
1766     transition.setTransitionCode(code);
1767     if (code == TransitionCode.OPENED && openSeqNum >= 0) {
1768       transition.setOpenSeqNum(openSeqNum);
1769     }
1770     for (HRegionInfo hri: hris) {
1771       transition.addRegionInfo(HRegionInfo.convert(hri));
1772     }
1773     ReportRegionStateTransitionRequest request = builder.build();
1774     while (keepLooping()) {
1775       RegionServerStatusService.BlockingInterface rss = rssStub;
1776       try {
1777         if (rss == null) {
1778           createRegionServerStatusStub();
1779           continue;
1780         }
1781         ReportRegionStateTransitionResponse response =
1782           rss.reportRegionStateTransition(null, request);
1783         if (response.hasErrorMessage()) {
1784           LOG.info("Failed to transition " + hris[0]
1785             + " to " + code + ": " + response.getErrorMessage());
1786           return false;
1787         }
1788         return true;
1789       } catch (ServiceException se) {
1790         IOException ioe = ProtobufUtil.getRemoteException(se);
1791         LOG.info("Failed to report region transition, will retry", ioe);
1792         if (rssStub == rss) {
1793           rssStub = null;
1794         }
1795       }
1796     }
1797     return false;
1798   }
1799 
1800   @Override
1801   public RpcServerInterface getRpcServer() {
1802     return rpcServices.rpcServer;
1803   }
1804 
1805   @VisibleForTesting
1806   public RSRpcServices getRSRpcServices() {
1807     return rpcServices;
1808   }
1809 
1810   /**
1811    * Cause the server to exit without closing the regions it is serving, the log
1812    * it is using and without notifying the master. Used unit testing and on
1813    * catastrophic events such as HDFS is yanked out from under hbase or we OOME.
1814    *
1815    * @param reason
1816    *          the reason we are aborting
1817    * @param cause
1818    *          the exception that caused the abort, or null
1819    */
1820   @Override
1821   public void abort(String reason, Throwable cause) {
1822     String msg = "ABORTING region server " + this + ": " + reason;
1823     if (cause != null) {
1824       LOG.fatal(msg, cause);
1825     } else {
1826       LOG.fatal(msg);
1827     }
1828     this.abortRequested = true;
1829     // HBASE-4014: show list of coprocessors that were loaded to help debug
1830     // regionserver crashes.Note that we're implicitly using
1831     // java.util.HashSet's toString() method to print the coprocessor names.
1832     LOG.fatal("RegionServer abort: loaded coprocessors are: " +
1833         CoprocessorHost.getLoadedCoprocessors());
1834     // Do our best to report our abort to the master, but this may not work
1835     try {
1836       if (cause != null) {
1837         msg += "\nCause:\n" + StringUtils.stringifyException(cause);
1838       }
1839       // Report to the master but only if we have already registered with the master.
1840       if (rssStub != null && this.serverName != null) {
1841         ReportRSFatalErrorRequest.Builder builder =
1842           ReportRSFatalErrorRequest.newBuilder();
1843         ServerName sn =
1844           ServerName.parseVersionedServerName(this.serverName.getVersionedBytes());
1845         builder.setServer(ProtobufUtil.toServerName(sn));
1846         builder.setErrorMessage(msg);
1847         rssStub.reportRSFatalError(null, builder.build());
1848       }
1849     } catch (Throwable t) {
1850       LOG.warn("Unable to report fatal error to master", t);
1851     }
1852     stop(reason);
1853   }
1854 
1855   /**
1856    * @see HRegionServer#abort(String, Throwable)
1857    */
1858   public void abort(String reason) {
1859     abort(reason, null);
1860   }
1861 
1862   @Override
1863   public boolean isAborted() {
1864     return this.abortRequested;
1865   }
1866 
1867   /*
1868    * Simulate a kill -9 of this server. Exits w/o closing regions or cleaninup
1869    * logs but it does close socket in case want to bring up server on old
1870    * hostname+port immediately.
1871    */
1872   protected void kill() {
1873     this.killed = true;
1874     abort("Simulated kill");
1875   }
1876 
1877   /**
1878    * Wait on all threads to finish. Presumption is that all closes and stops
1879    * have already been called.
1880    */
1881   protected void stopServiceThreads() {
1882     if (this.nonceManagerChore != null) {
1883       Threads.shutdown(this.nonceManagerChore.getThread());
1884     }
1885     if (this.compactionChecker != null) {
1886       Threads.shutdown(this.compactionChecker.getThread());
1887     }
1888     if (this.periodicFlusher != null) {
1889       Threads.shutdown(this.periodicFlusher.getThread());
1890     }
1891     if (this.cacheFlusher != null) {
1892       this.cacheFlusher.join();
1893     }
1894     if (this.healthCheckChore != null) {
1895       Threads.shutdown(this.healthCheckChore.getThread());
1896     }
1897     if (this.spanReceiverHost != null) {
1898       this.spanReceiverHost.closeReceivers();
1899     }
1900     if (this.hlogRoller != null) {
1901       Threads.shutdown(this.hlogRoller.getThread());
1902     }
1903     if (this.metaHLogRoller != null) {
1904       Threads.shutdown(this.metaHLogRoller.getThread());
1905     }
1906     if (this.compactSplitThread != null) {
1907       this.compactSplitThread.join();
1908     }
1909     if (this.service != null) this.service.shutdown();
1910     if (this.replicationSourceHandler != null &&
1911         this.replicationSourceHandler == this.replicationSinkHandler) {
1912       this.replicationSourceHandler.stopReplicationService();
1913     } else {
1914       if (this.replicationSourceHandler != null) {
1915         this.replicationSourceHandler.stopReplicationService();
1916       }
1917       if (this.replicationSinkHandler != null) {
1918         this.replicationSinkHandler.stopReplicationService();
1919       }
1920     }
1921     if (this.storefileRefresher != null) {
1922       Threads.shutdown(this.storefileRefresher.getThread());
1923     }
1924   }
1925 
1926   /**
1927    * @return Return the object that implements the replication
1928    * source service.
1929    */
1930   ReplicationSourceService getReplicationSourceService() {
1931     return replicationSourceHandler;
1932   }
1933 
1934   /**
1935    * @return Return the object that implements the replication
1936    * sink service.
1937    */
1938   ReplicationSinkService getReplicationSinkService() {
1939     return replicationSinkHandler;
1940   }
1941 
1942   /**
1943    * Get the current master from ZooKeeper and open the RPC connection to it.
1944    *
1945    * Method will block until a master is available. You can break from this
1946    * block by requesting the server stop.
1947    *
1948    * @return master + port, or null if server has been stopped
1949    */
1950   private synchronized ServerName createRegionServerStatusStub() {
1951     if (rssStub != null) {
1952       return masterAddressTracker.getMasterAddress();
1953     }
1954     ServerName sn = null;
1955     long previousLogTime = 0;
1956     RegionServerStatusService.BlockingInterface master = null;
1957     boolean refresh = false; // for the first time, use cached data
1958     RegionServerStatusService.BlockingInterface intf = null;
1959     boolean interrupted = false;
1960     try {
1961       while (keepLooping() && master == null) {
1962         sn = this.masterAddressTracker.getMasterAddress(refresh);
1963         if (sn == null) {
1964           if (!keepLooping()) {
1965             // give up with no connection.
1966             LOG.debug("No master found and cluster is stopped; bailing out");
1967             return null;
1968           }
1969           LOG.debug("No master found; retry");
1970           previousLogTime = System.currentTimeMillis();
1971           refresh = true; // let's try pull it from ZK directly
1972           sleeper.sleep();
1973           continue;
1974         }
1975 
1976         // If we are on the active master, use the shortcut
1977         if (this instanceof HMaster && sn.equals(getServerName())) {
1978           intf = ((HMaster)this).getMasterRpcServices();
1979           break;
1980         }
1981         try {
1982           BlockingRpcChannel channel =
1983             this.rpcClient.createBlockingRpcChannel(sn, userProvider.getCurrent(), operationTimeout);
1984           intf = RegionServerStatusService.newBlockingStub(channel);
1985           break;
1986         } catch (IOException e) {
1987           e = e instanceof RemoteException ?
1988             ((RemoteException)e).unwrapRemoteException() : e;
1989           if (e instanceof ServerNotRunningYetException) {
1990             if (System.currentTimeMillis() > (previousLogTime+1000)){
1991               LOG.info("Master isn't available yet, retrying");
1992               previousLogTime = System.currentTimeMillis();
1993             }
1994           } else {
1995             if (System.currentTimeMillis() > (previousLogTime + 1000)) {
1996               LOG.warn("Unable to connect to master. Retrying. Error was:", e);
1997               previousLogTime = System.currentTimeMillis();
1998             }
1999           }
2000           try {
2001             Thread.sleep(200);
2002           } catch (InterruptedException ex) {
2003             interrupted = true;
2004             LOG.warn("Interrupted while sleeping");
2005           }
2006         }
2007       }
2008     } finally {
2009       if (interrupted) {
2010         Thread.currentThread().interrupt();
2011       }
2012     }
2013     rssStub = intf;
2014     return sn;
2015   }
2016 
2017   /**
2018    * @return True if we should break loop because cluster is going down or
2019    * this server has been stopped or hdfs has gone bad.
2020    */
2021   private boolean keepLooping() {
2022     return !this.stopped && isClusterUp();
2023   }
2024 
2025   /*
2026    * Let the master know we're here Run initialization using parameters passed
2027    * us by the master.
2028    * @return A Map of key/value configurations we got from the Master else
2029    * null if we failed to register.
2030    * @throws IOException
2031    */
2032   private RegionServerStartupResponse reportForDuty() throws IOException {
2033     ServerName masterServerName = createRegionServerStatusStub();
2034     if (masterServerName == null) return null;
2035     RegionServerStartupResponse result = null;
2036     try {
2037       rpcServices.requestCount.set(0);
2038       LOG.info("reportForDuty to master=" + masterServerName + " with port="
2039         + rpcServices.isa.getPort() + ", startcode=" + this.startcode);
2040       long now = EnvironmentEdgeManager.currentTimeMillis();
2041       int port = rpcServices.isa.getPort();
2042       RegionServerStartupRequest.Builder request = RegionServerStartupRequest.newBuilder();
2043       request.setPort(port);
2044       request.setServerStartCode(this.startcode);
2045       request.setServerCurrentTime(now);
2046       result = this.rssStub.regionServerStartup(null, request.build());
2047     } catch (ServiceException se) {
2048       IOException ioe = ProtobufUtil.getRemoteException(se);
2049       if (ioe instanceof ClockOutOfSyncException) {
2050         LOG.fatal("Master rejected startup because clock is out of sync", ioe);
2051         // Re-throw IOE will cause RS to abort
2052         throw ioe;
2053       } else if (ioe instanceof ServerNotRunningYetException) {
2054         LOG.debug("Master is not running yet");
2055       } else {
2056         LOG.warn("error telling master we are up", se);
2057       }
2058     }
2059     return result;
2060   }
2061 
2062   @Override
2063   public long getLastSequenceId(byte[] region) {
2064     Long lastFlushedSequenceId = -1l;
2065     try {
2066       GetLastFlushedSequenceIdRequest req = RequestConverter
2067           .buildGetLastFlushedSequenceIdRequest(region);
2068       lastFlushedSequenceId = rssStub.getLastFlushedSequenceId(null, req)
2069           .getLastFlushedSequenceId();
2070     } catch (ServiceException e) {
2071       lastFlushedSequenceId = -1l;
2072       LOG.warn("Unable to connect to the master to check " + "the last flushed sequence id", e);
2073     }
2074     return lastFlushedSequenceId;
2075   }
2076 
2077   /**
2078    * Closes all regions.  Called on our way out.
2079    * Assumes that its not possible for new regions to be added to onlineRegions
2080    * while this method runs.
2081    */
2082   protected void closeAllRegions(final boolean abort) {
2083     closeUserRegions(abort);
2084     closeMetaTableRegions(abort);
2085   }
2086 
2087   /**
2088    * Close meta region if we carry it
2089    * @param abort Whether we're running an abort.
2090    */
2091   void closeMetaTableRegions(final boolean abort) {
2092     HRegion meta = null;
2093     this.lock.writeLock().lock();
2094     try {
2095       for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
2096         HRegionInfo hri = e.getValue().getRegionInfo();
2097         if (hri.isMetaRegion()) {
2098           meta = e.getValue();
2099         }
2100         if (meta != null) break;
2101       }
2102     } finally {
2103       this.lock.writeLock().unlock();
2104     }
2105     if (meta != null) closeRegionIgnoreErrors(meta.getRegionInfo(), abort);
2106   }
2107 
2108   /**
2109    * Schedule closes on all user regions.
2110    * Should be safe calling multiple times because it wont' close regions
2111    * that are already closed or that are closing.
2112    * @param abort Whether we're running an abort.
2113    */
2114   void closeUserRegions(final boolean abort) {
2115     this.lock.writeLock().lock();
2116     try {
2117       for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
2118         HRegion r = e.getValue();
2119         if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
2120           // Don't update zk with this close transition; pass false.
2121           closeRegionIgnoreErrors(r.getRegionInfo(), abort);
2122         }
2123       }
2124     } finally {
2125       this.lock.writeLock().unlock();
2126     }
2127   }
2128 
2129   /** @return the info server */
2130   public InfoServer getInfoServer() {
2131     return infoServer;
2132   }
2133 
2134   /**
2135    * @return true if a stop has been requested.
2136    */
2137   @Override
2138   public boolean isStopped() {
2139     return this.stopped;
2140   }
2141 
2142   @Override
2143   public boolean isStopping() {
2144     return this.stopping;
2145   }
2146 
2147   @Override
2148   public Map<String, HRegion> getRecoveringRegions() {
2149     return this.recoveringRegions;
2150   }
2151 
2152   /**
2153    *
2154    * @return the configuration
2155    */
2156   @Override
2157   public Configuration getConfiguration() {
2158     return conf;
2159   }
2160 
2161   /** @return the write lock for the server */
2162   ReentrantReadWriteLock.WriteLock getWriteLock() {
2163     return lock.writeLock();
2164   }
2165 
2166   public int getNumberOfOnlineRegions() {
2167     return this.onlineRegions.size();
2168   }
2169 
2170   boolean isOnlineRegionsEmpty() {
2171     return this.onlineRegions.isEmpty();
2172   }
2173 
2174   /**
2175    * For tests, web ui and metrics.
2176    * This method will only work if HRegionServer is in the same JVM as client;
2177    * HRegion cannot be serialized to cross an rpc.
2178    */
2179   public Collection<HRegion> getOnlineRegionsLocalContext() {
2180     Collection<HRegion> regions = this.onlineRegions.values();
2181     return Collections.unmodifiableCollection(regions);
2182   }
2183 
2184   @Override
2185   public void addToOnlineRegions(HRegion region) {
2186     this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
2187   }
2188 
2189   /**
2190    * @return A new Map of online regions sorted by region size with the first entry being the
2191    * biggest.  If two regions are the same size, then the last one found wins; i.e. this method
2192    * may NOT return all regions.
2193    */
2194   SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
2195     // we'll sort the regions in reverse
2196     SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
2197         new Comparator<Long>() {
2198           @Override
2199           public int compare(Long a, Long b) {
2200             return -1 * a.compareTo(b);
2201           }
2202         });
2203     // Copy over all regions. Regions are sorted by size with biggest first.
2204     for (HRegion region : this.onlineRegions.values()) {
2205       sortedRegions.put(region.memstoreSize.get(), region);
2206     }
2207     return sortedRegions;
2208   }
2209 
2210   /**
2211    * @return time stamp in millis of when this region server was started
2212    */
2213   public long getStartcode() {
2214     return this.startcode;
2215   }
2216 
2217   /** @return reference to FlushRequester */
2218   @Override
2219   public FlushRequester getFlushRequester() {
2220     return this.cacheFlusher;
2221   }
2222 
2223   /**
2224    * Get the top N most loaded regions this server is serving so we can tell the
2225    * master which regions it can reallocate if we're overloaded. TODO: actually
2226    * calculate which regions are most loaded. (Right now, we're just grabbing
2227    * the first N regions being served regardless of load.)
2228    */
2229   protected HRegionInfo[] getMostLoadedRegions() {
2230     ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
2231     for (HRegion r : onlineRegions.values()) {
2232       if (!r.isAvailable()) {
2233         continue;
2234       }
2235       if (regions.size() < numRegionsToReport) {
2236         regions.add(r.getRegionInfo());
2237       } else {
2238         break;
2239       }
2240     }
2241     return regions.toArray(new HRegionInfo[regions.size()]);
2242   }
2243 
2244   @Override
2245   public Leases getLeases() {
2246     return leases;
2247   }
2248 
2249   /**
2250    * @return Return the rootDir.
2251    */
2252   protected Path getRootDir() {
2253     return rootDir;
2254   }
2255 
2256   /**
2257    * @return Return the fs.
2258    */
2259   @Override
2260   public FileSystem getFileSystem() {
2261     return fs;
2262   }
2263 
2264   @Override
2265   public String toString() {
2266     return getServerName().toString();
2267   }
2268 
2269   /**
2270    * Interval at which threads should run
2271    *
2272    * @return the interval
2273    */
2274   public int getThreadWakeFrequency() {
2275     return threadWakeFrequency;
2276   }
2277 
2278   @Override
2279   public ZooKeeperWatcher getZooKeeper() {
2280     return zooKeeper;
2281   }
2282 
2283   @Override
2284   public BaseCoordinatedStateManager getCoordinatedStateManager() {
2285     return csm;
2286   }
2287 
2288   @Override
2289   public ServerName getServerName() {
2290     return serverName;
2291   }
2292 
2293   @Override
2294   public CompactionRequestor getCompactionRequester() {
2295     return this.compactSplitThread;
2296   }
2297 
2298   public RegionServerCoprocessorHost getRegionServerCoprocessorHost(){
2299     return this.rsHost;
2300   }
2301 
2302   @Override
2303   public ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS() {
2304     return this.regionsInTransitionInRS;
2305   }
2306 
2307   @Override
2308   public ExecutorService getExecutorService() {
2309     return service;
2310   }
2311 
2312   //
2313   // Main program and support routines
2314   //
2315 
2316   /**
2317    * Load the replication service objects, if any
2318    */
2319   static private void createNewReplicationInstance(Configuration conf,
2320     HRegionServer server, FileSystem fs, Path logDir, Path oldLogDir) throws IOException{
2321 
2322     // If replication is not enabled, then return immediately.
2323     if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
2324         HConstants.REPLICATION_ENABLE_DEFAULT)) {
2325       return;
2326     }
2327 
2328     // read in the name of the source replication class from the config file.
2329     String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
2330                                HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
2331 
2332     // read in the name of the sink replication class from the config file.
2333     String sinkClassname = conf.get(HConstants.REPLICATION_SINK_SERVICE_CLASSNAME,
2334                              HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
2335 
2336     // If both the sink and the source class names are the same, then instantiate
2337     // only one object.
2338     if (sourceClassname.equals(sinkClassname)) {
2339       server.replicationSourceHandler = (ReplicationSourceService)
2340                                          newReplicationInstance(sourceClassname,
2341                                          conf, server, fs, logDir, oldLogDir);
2342       server.replicationSinkHandler = (ReplicationSinkService)
2343                                          server.replicationSourceHandler;
2344     } else {
2345       server.replicationSourceHandler = (ReplicationSourceService)
2346                                          newReplicationInstance(sourceClassname,
2347                                          conf, server, fs, logDir, oldLogDir);
2348       server.replicationSinkHandler = (ReplicationSinkService)
2349                                          newReplicationInstance(sinkClassname,
2350                                          conf, server, fs, logDir, oldLogDir);
2351     }
2352   }
2353 
2354   static private ReplicationService newReplicationInstance(String classname,
2355     Configuration conf, HRegionServer server, FileSystem fs, Path logDir,
2356     Path oldLogDir) throws IOException{
2357 
2358     Class<?> clazz = null;
2359     try {
2360       ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
2361       clazz = Class.forName(classname, true, classLoader);
2362     } catch (java.lang.ClassNotFoundException nfe) {
2363       throw new IOException("Could not find class for " + classname);
2364     }
2365 
2366     // create an instance of the replication object.
2367     ReplicationService service = (ReplicationService)
2368                               ReflectionUtils.newInstance(clazz, conf);
2369     service.initialize(server, fs, logDir, oldLogDir);
2370     return service;
2371   }
2372 
2373   /**
2374    * Utility for constructing an instance of the passed HRegionServer class.
2375    *
2376    * @param regionServerClass
2377    * @param conf2
2378    * @return HRegionServer instance.
2379    */
2380   public static HRegionServer constructRegionServer(
2381       Class<? extends HRegionServer> regionServerClass,
2382       final Configuration conf2, CoordinatedStateManager cp) {
2383     try {
2384       Constructor<? extends HRegionServer> c = regionServerClass
2385           .getConstructor(Configuration.class, CoordinatedStateManager.class);
2386       return c.newInstance(conf2, cp);
2387     } catch (Exception e) {
2388       throw new RuntimeException("Failed construction of " + "Regionserver: "
2389           + regionServerClass.toString(), e);
2390     }
2391   }
2392 
2393   /**
2394    * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
2395    */
2396   public static void main(String[] args) throws Exception {
2397 	VersionInfo.logVersion();
2398     Configuration conf = HBaseConfiguration.create();
2399     @SuppressWarnings("unchecked")
2400     Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
2401         .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
2402 
2403     new HRegionServerCommandLine(regionServerClass).doMain(args);
2404   }
2405 
2406   /**
2407    * Gets the online regions of the specified table.
2408    * This method looks at the in-memory onlineRegions.  It does not go to <code>hbase:meta</code>.
2409    * Only returns <em>online</em> regions.  If a region on this table has been
2410    * closed during a disable, etc., it will not be included in the returned list.
2411    * So, the returned list may not necessarily be ALL regions in this table, its
2412    * all the ONLINE regions in the table.
2413    * @param tableName
2414    * @return Online regions from <code>tableName</code>
2415    */
2416   @Override
2417   public List<HRegion> getOnlineRegions(TableName tableName) {
2418      List<HRegion> tableRegions = new ArrayList<HRegion>();
2419      synchronized (this.onlineRegions) {
2420        for (HRegion region: this.onlineRegions.values()) {
2421          HRegionInfo regionInfo = region.getRegionInfo();
2422          if(regionInfo.getTable().equals(tableName)) {
2423            tableRegions.add(region);
2424          }
2425        }
2426      }
2427      return tableRegions;
2428    }
2429 
2430   // used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
2431   public String[] getRegionServerCoprocessors() {
2432     TreeSet<String> coprocessors = new TreeSet<String>(
2433         this.hlog.getCoprocessorHost().getCoprocessors());
2434     Collection<HRegion> regions = getOnlineRegionsLocalContext();
2435     for (HRegion region: regions) {
2436       coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
2437     }
2438     return coprocessors.toArray(new String[coprocessors.size()]);
2439   }
2440 
2441   /**
2442    * Try to close the region, logs a warning on failure but continues.
2443    * @param region Region to close
2444    */
2445   private void closeRegionIgnoreErrors(HRegionInfo region, final boolean abort) {
2446     try {
2447       if (!closeRegion(region.getEncodedName(), abort, null)) {
2448         LOG.warn("Failed to close " + region.getRegionNameAsString() +
2449             " - ignoring and continuing");
2450       }
2451     } catch (IOException e) {
2452       LOG.warn("Failed to close " + region.getRegionNameAsString() +
2453           " - ignoring and continuing", e);
2454     }
2455   }
2456 
2457   /**
2458    * Close asynchronously a region, can be called from the master or internally by the regionserver
2459    * when stopping. If called from the master, the region will update the znode status.
2460    *
2461    * <p>
2462    * If an opening was in progress, this method will cancel it, but will not start a new close. The
2463    * coprocessors are not called in this case. A NotServingRegionException exception is thrown.
2464    * </p>
2465 
2466    * <p>
2467    *   If a close was in progress, this new request will be ignored, and an exception thrown.
2468    * </p>
2469    *
2470    * @param encodedName Region to close
2471    * @param abort True if we are aborting
2472    * @return True if closed a region.
2473    * @throws NotServingRegionException if the region is not online
2474    * @throws RegionAlreadyInTransitionException if the region is already closing
2475    */
2476   protected boolean closeRegion(String encodedName, final boolean abort, final ServerName sn)
2477       throws NotServingRegionException, RegionAlreadyInTransitionException {
2478     //Check for permissions to close.
2479     HRegion actualRegion = this.getFromOnlineRegions(encodedName);
2480     if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
2481       try {
2482         actualRegion.getCoprocessorHost().preClose(false);
2483       } catch (IOException exp) {
2484         LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
2485         return false;
2486       }
2487     }
2488 
2489     final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(encodedName.getBytes(),
2490         Boolean.FALSE);
2491 
2492     if (Boolean.TRUE.equals(previous)) {
2493       LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
2494           "trying to OPEN. Cancelling OPENING.");
2495       if (!regionsInTransitionInRS.replace(encodedName.getBytes(), previous, Boolean.FALSE)){
2496         // The replace failed. That should be an exceptional case, but theoretically it can happen.
2497         // We're going to try to do a standard close then.
2498         LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
2499             " Doing a standard close now");
2500         return closeRegion(encodedName, abort, sn);
2501       }
2502       // Let's get the region from the online region list again
2503       actualRegion = this.getFromOnlineRegions(encodedName);
2504       if (actualRegion == null) { // If already online, we still need to close it.
2505         LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
2506         // The master deletes the znode when it receives this exception.
2507         throw new NotServingRegionException("The region " + encodedName +
2508           " was opening but not yet served. Opening is cancelled.");
2509       }
2510     } else if (Boolean.FALSE.equals(previous)) {
2511       LOG.info("Received CLOSE for the region: " + encodedName +
2512         " ,which we are already trying to CLOSE, but not completed yet");
2513       // The master will retry till the region is closed. We need to do this since
2514       // the region could fail to close somehow. If we mark the region closed in master
2515       // while it is not, there could be data loss.
2516       // If the region stuck in closing for a while, and master runs out of retries,
2517       // master will move the region to failed_to_close. Later on, if the region
2518       // is indeed closed, master can properly re-assign it.
2519       throw new RegionAlreadyInTransitionException("The region " + encodedName +
2520         " was already closing. New CLOSE request is ignored.");
2521     }
2522 
2523     if (actualRegion == null) {
2524       LOG.error("Received CLOSE for a region which is not online, and we're not opening.");
2525       this.regionsInTransitionInRS.remove(encodedName.getBytes());
2526       // The master deletes the znode when it receives this exception.
2527       throw new NotServingRegionException("The region " + encodedName +
2528           " is not online, and is not opening.");
2529     }
2530 
2531     CloseRegionHandler crh;
2532     final HRegionInfo hri = actualRegion.getRegionInfo();
2533     if (hri.isMetaRegion()) {
2534       crh = new CloseMetaHandler(this, this, hri, abort);
2535     } else {
2536       crh = new CloseRegionHandler(this, this, hri, abort, sn);
2537     }
2538     this.service.submit(crh);
2539     return true;
2540   }
2541 
2542    /**
2543    * @param regionName
2544    * @return HRegion for the passed binary <code>regionName</code> or null if
2545    *         named region is not member of the online regions.
2546    */
2547   public HRegion getOnlineRegion(final byte[] regionName) {
2548     String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
2549     return this.onlineRegions.get(encodedRegionName);
2550   }
2551 
2552   public InetSocketAddress[] getRegionBlockLocations(final String encodedRegionName) {
2553     return this.regionFavoredNodesMap.get(encodedRegionName);
2554   }
2555 
2556   @Override
2557   public HRegion getFromOnlineRegions(final String encodedRegionName) {
2558     return this.onlineRegions.get(encodedRegionName);
2559   }
2560 
2561 
2562   @Override
2563   public boolean removeFromOnlineRegions(final HRegion r, ServerName destination) {
2564     HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
2565 
2566     if (destination != null) {
2567       HLog wal = getWAL();
2568       long closeSeqNum = wal.getEarliestMemstoreSeqNum(r.getRegionInfo().getEncodedNameAsBytes());
2569       if (closeSeqNum == HConstants.NO_SEQNUM) {
2570         // No edits in WAL for this region; get the sequence number when the region was opened.
2571         closeSeqNum = r.getOpenSeqNum();
2572         if (closeSeqNum == HConstants.NO_SEQNUM) {
2573           closeSeqNum = 0;
2574         }
2575       }
2576       addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum);
2577     }
2578     this.regionFavoredNodesMap.remove(r.getRegionInfo().getEncodedName());
2579     return toReturn != null;
2580   }
2581 
2582   /**
2583    * Protected utility method for safely obtaining an HRegion handle.
2584    *
2585    * @param regionName
2586    *          Name of online {@link HRegion} to return
2587    * @return {@link HRegion} for <code>regionName</code>
2588    * @throws NotServingRegionException
2589    */
2590   protected HRegion getRegion(final byte[] regionName)
2591       throws NotServingRegionException {
2592     String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
2593     return getRegionByEncodedName(regionName, encodedRegionName);
2594   }
2595 
2596   public HRegion getRegionByEncodedName(String encodedRegionName)
2597       throws NotServingRegionException {
2598     return getRegionByEncodedName(null, encodedRegionName);
2599   }
2600 
2601   protected HRegion getRegionByEncodedName(byte[] regionName, String encodedRegionName)
2602     throws NotServingRegionException {
2603     HRegion region = this.onlineRegions.get(encodedRegionName);
2604     if (region == null) {
2605       MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
2606       if (moveInfo != null) {
2607         throw new RegionMovedException(moveInfo.getServerName(), moveInfo.getSeqNum());
2608       }
2609       Boolean isOpening = this.regionsInTransitionInRS.get(Bytes.toBytes(encodedRegionName));
2610       String regionNameStr = regionName == null?
2611         encodedRegionName: Bytes.toStringBinary(regionName);
2612       if (isOpening != null && isOpening.booleanValue()) {
2613         throw new RegionOpeningException("Region " + regionNameStr +
2614           " is opening on " + this.serverName);
2615       }
2616       throw new NotServingRegionException("Region " + regionNameStr +
2617         " is not online on " + this.serverName);
2618     }
2619     return region;
2620   }
2621 
2622   /*
2623    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
2624    * IOE if it isn't already.
2625    *
2626    * @param t Throwable
2627    *
2628    * @param msg Message to log in error. Can be null.
2629    *
2630    * @return Throwable converted to an IOE; methods can only let out IOEs.
2631    */
2632   private Throwable cleanup(final Throwable t, final String msg) {
2633     // Don't log as error if NSRE; NSRE is 'normal' operation.
2634     if (t instanceof NotServingRegionException) {
2635       LOG.debug("NotServingRegionException; " + t.getMessage());
2636       return t;
2637     }
2638     Throwable e = t instanceof RemoteException ? ((RemoteException) t).unwrapRemoteException() : t;
2639     if (msg == null) {
2640       LOG.error("", e);
2641     } else {
2642       LOG.error(msg, e);
2643     }
2644     if (!rpcServices.checkOOME(t)) {
2645       checkFileSystem();
2646     }
2647     return t;
2648   }
2649 
2650   /*
2651    * @param t
2652    *
2653    * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
2654    *
2655    * @return Make <code>t</code> an IOE if it isn't already.
2656    */
2657   protected IOException convertThrowableToIOE(final Throwable t, final String msg) {
2658     return (t instanceof IOException ? (IOException) t : msg == null
2659         || msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
2660   }
2661 
2662   /**
2663    * Checks to see if the file system is still accessible. If not, sets
2664    * abortRequested and stopRequested
2665    *
2666    * @return false if file system is not available
2667    */
2668   public boolean checkFileSystem() {
2669     if (this.fsOk && this.fs != null) {
2670       try {
2671         FSUtils.checkFileSystemAvailable(this.fs);
2672       } catch (IOException e) {
2673         abort("File System not available", e);
2674         this.fsOk = false;
2675       }
2676     }
2677     return this.fsOk;
2678   }
2679 
2680   @Override
2681   public void updateRegionFavoredNodesMapping(String encodedRegionName,
2682       List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName> favoredNodes) {
2683     InetSocketAddress[] addr = new InetSocketAddress[favoredNodes.size()];
2684     // Refer to the comment on the declaration of regionFavoredNodesMap on why
2685     // it is a map of region name to InetSocketAddress[]
2686     for (int i = 0; i < favoredNodes.size(); i++) {
2687       addr[i] = InetSocketAddress.createUnresolved(favoredNodes.get(i).getHostName(),
2688           favoredNodes.get(i).getPort());
2689     }
2690     regionFavoredNodesMap.put(encodedRegionName, addr);
2691   }
2692 
2693   /**
2694    * Return the favored nodes for a region given its encoded name. Look at the
2695    * comment around {@link #regionFavoredNodesMap} on why it is InetSocketAddress[]
2696    * @param encodedRegionName
2697    * @return array of favored locations
2698    */
2699   @Override
2700   public InetSocketAddress[] getFavoredNodesForRegion(String encodedRegionName) {
2701     return regionFavoredNodesMap.get(encodedRegionName);
2702   }
2703 
2704   @Override
2705   public ServerNonceManager getNonceManager() {
2706     return this.nonceManager;
2707   }
2708 
2709   private static class MovedRegionInfo {
2710     private final ServerName serverName;
2711     private final long seqNum;
2712     private final long ts;
2713 
2714     public MovedRegionInfo(ServerName serverName, long closeSeqNum) {
2715       this.serverName = serverName;
2716       this.seqNum = closeSeqNum;
2717       ts = EnvironmentEdgeManager.currentTimeMillis();
2718      }
2719 
2720     public ServerName getServerName() {
2721       return serverName;
2722     }
2723 
2724     public long getSeqNum() {
2725       return seqNum;
2726     }
2727 
2728     public long getMoveTime() {
2729       return ts;
2730     }
2731   }
2732 
2733   // This map will contains all the regions that we closed for a move.
2734   //  We add the time it was moved as we don't want to keep too old information
2735   protected Map<String, MovedRegionInfo> movedRegions =
2736       new ConcurrentHashMap<String, MovedRegionInfo>(3000);
2737 
2738   // We need a timeout. If not there is a risk of giving a wrong information: this would double
2739   //  the number of network calls instead of reducing them.
2740   private static final int TIMEOUT_REGION_MOVED = (2 * 60 * 1000);
2741 
2742   protected void addToMovedRegions(String encodedName, ServerName destination, long closeSeqNum) {
2743     if (ServerName.isSameHostnameAndPort(destination, this.getServerName())) {
2744       LOG.warn("Not adding moved region record: " + encodedName + " to self.");
2745       return;
2746     }
2747     LOG.info("Adding moved region record: "
2748       + encodedName + " to " + destination + " as of " + closeSeqNum);
2749     movedRegions.put(encodedName, new MovedRegionInfo(destination, closeSeqNum));
2750   }
2751 
2752   void removeFromMovedRegions(String encodedName) {
2753     movedRegions.remove(encodedName);
2754   }
2755 
2756   private MovedRegionInfo getMovedRegion(final String encodedRegionName) {
2757     MovedRegionInfo dest = movedRegions.get(encodedRegionName);
2758 
2759     long now = EnvironmentEdgeManager.currentTimeMillis();
2760     if (dest != null) {
2761       if (dest.getMoveTime() > (now - TIMEOUT_REGION_MOVED)) {
2762         return dest;
2763       } else {
2764         movedRegions.remove(encodedRegionName);
2765       }
2766     }
2767 
2768     return null;
2769   }
2770 
2771   /**
2772    * Remove the expired entries from the moved regions list.
2773    */
2774   protected void cleanMovedRegions() {
2775     final long cutOff = System.currentTimeMillis() - TIMEOUT_REGION_MOVED;
2776     Iterator<Entry<String, MovedRegionInfo>> it = movedRegions.entrySet().iterator();
2777 
2778     while (it.hasNext()){
2779       Map.Entry<String, MovedRegionInfo> e = it.next();
2780       if (e.getValue().getMoveTime() < cutOff) {
2781         it.remove();
2782       }
2783     }
2784   }
2785 
2786   /**
2787    * Creates a Chore thread to clean the moved region cache.
2788    */
2789   protected static class MovedRegionsCleaner extends Chore implements Stoppable {
2790     private HRegionServer regionServer;
2791     Stoppable stoppable;
2792 
2793     private MovedRegionsCleaner(
2794       HRegionServer regionServer, Stoppable stoppable){
2795       super("MovedRegionsCleaner for region "+regionServer, TIMEOUT_REGION_MOVED, stoppable);
2796       this.regionServer = regionServer;
2797       this.stoppable = stoppable;
2798     }
2799 
2800     static MovedRegionsCleaner createAndStart(HRegionServer rs){
2801       Stoppable stoppable = new Stoppable() {
2802         private volatile boolean isStopped = false;
2803         @Override public void stop(String why) { isStopped = true;}
2804         @Override public boolean isStopped() {return isStopped;}
2805       };
2806 
2807       return new MovedRegionsCleaner(rs, stoppable);
2808     }
2809 
2810     @Override
2811     protected void chore() {
2812       regionServer.cleanMovedRegions();
2813     }
2814 
2815     @Override
2816     public void stop(String why) {
2817       stoppable.stop(why);
2818     }
2819 
2820     @Override
2821     public boolean isStopped() {
2822       return stoppable.isStopped();
2823     }
2824   }
2825 
2826   private String getMyEphemeralNodePath() {
2827     return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString());
2828   }
2829 
2830   private boolean isHealthCheckerConfigured() {
2831     String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
2832     return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
2833   }
2834 
2835   /**
2836    * @return the underlying {@link CompactSplitThread} for the servers
2837    */
2838   public CompactSplitThread getCompactSplitThread() {
2839     return this.compactSplitThread;
2840   }
2841 
2842   /**
2843    * A helper function to store the last flushed sequence Id with the previous failed RS for a
2844    * recovering region. The Id is used to skip wal edits which are flushed. Since the flushed
2845    * sequence id is only valid for each RS, we associate the Id with corresponding failed RS.
2846    * @throws KeeperException
2847    * @throws IOException
2848    */
2849   private void updateRecoveringRegionLastFlushedSequenceId(HRegion r) throws KeeperException,
2850       IOException {
2851     if (!r.isRecovering()) {
2852       // return immdiately for non-recovering regions
2853       return;
2854     }
2855 
2856     HRegionInfo region = r.getRegionInfo();
2857     ZooKeeperWatcher zkw = getZooKeeper();
2858     String previousRSName = this.getLastFailedRSFromZK(region.getEncodedName());
2859     Map<byte[], Long> maxSeqIdInStores = r.getMaxStoreSeqIdForLogReplay();
2860     long minSeqIdForLogReplay = -1;
2861     for (Long storeSeqIdForReplay : maxSeqIdInStores.values()) {
2862       if (minSeqIdForLogReplay == -1 || storeSeqIdForReplay < minSeqIdForLogReplay) {
2863         minSeqIdForLogReplay = storeSeqIdForReplay;
2864       }
2865     }
2866     
2867     try {
2868       long lastRecordedFlushedSequenceId = -1;
2869       String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode,
2870         region.getEncodedName());
2871       // recovering-region level
2872       byte[] data;
2873       try {
2874         data = ZKUtil.getData(zkw, nodePath);
2875       } catch (InterruptedException e) {
2876         throw new InterruptedIOException();
2877       }
2878       if (data != null) {
2879         lastRecordedFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data);
2880       }
2881       if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) {
2882         ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay));
2883       }
2884       if (previousRSName != null) {
2885         // one level deeper for the failed RS
2886         nodePath = ZKUtil.joinZNode(nodePath, previousRSName);
2887         ZKUtil.setData(zkw, nodePath,
2888           ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores));
2889         LOG.debug("Update last flushed sequence id of region " + region.getEncodedName() + " for "
2890             + previousRSName);
2891       } else {
2892         LOG.warn("Can't find failed region server for recovering region " + 
2893           region.getEncodedName());
2894       }
2895     } catch (NoNodeException ignore) {
2896       LOG.debug("Region " + region.getEncodedName() + 
2897         " must have completed recovery because its recovery znode has been removed", ignore);
2898     }
2899   }
2900 
2901   /**
2902    * Return the last failed RS name under /hbase/recovering-regions/encodedRegionName
2903    * @param encodedRegionName
2904    * @throws KeeperException
2905    */
2906   private String getLastFailedRSFromZK(String encodedRegionName) throws KeeperException {
2907     String result = null;
2908     long maxZxid = 0;
2909     ZooKeeperWatcher zkw = this.getZooKeeper();
2910     String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName);
2911     List<String> failedServers = ZKUtil.listChildrenNoWatch(zkw, nodePath);
2912     if (failedServers == null || failedServers.isEmpty()) {
2913       return result;
2914     }
2915     for (String failedServer : failedServers) {
2916       String rsPath = ZKUtil.joinZNode(nodePath, failedServer);
2917       Stat stat = new Stat();
2918       ZKUtil.getDataNoWatch(zkw, rsPath, stat);
2919       if (maxZxid < stat.getCzxid()) {
2920         maxZxid = stat.getCzxid();
2921         result = failedServer;
2922       }
2923     }
2924     return result;
2925   }
2926 
2927   /**
2928    * @return The cache config instance used by the regionserver.
2929    */
2930   public CacheConfig getCacheConfig() {
2931     return this.cacheConfig;
2932   }
2933 }