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.NotServingRegionException;
68  import org.apache.hadoop.hbase.RemoteExceptionHandler;
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.MetaTableAccessor;
76  import org.apache.hadoop.hbase.client.ConnectionUtils;
77  import org.apache.hadoop.hbase.client.HConnection;
78  import org.apache.hadoop.hbase.client.HConnectionManager;
79  import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
80  import org.apache.hadoop.hbase.coordination.CloseRegionCoordination;
81  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
82  import org.apache.hadoop.hbase.exceptions.RegionMovedException;
83  import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
84  import org.apache.hadoop.hbase.executor.ExecutorService;
85  import org.apache.hadoop.hbase.executor.ExecutorType;
86  import org.apache.hadoop.hbase.fs.HFileSystem;
87  import org.apache.hadoop.hbase.http.InfoServer;
88  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
89  import org.apache.hadoop.hbase.ipc.RpcClient;
90  import org.apache.hadoop.hbase.ipc.RpcServerInterface;
91  import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
92  import org.apache.hadoop.hbase.master.HMaster;
93  import org.apache.hadoop.hbase.master.SplitLogManager;
94  import org.apache.hadoop.hbase.master.TableLockManager;
95  import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
96  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
97  import org.apache.hadoop.hbase.protobuf.RequestConverter;
98  import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
99  import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
100 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.Coprocessor;
101 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
102 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
103 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
104 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
105 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
106 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
107 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
108 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
109 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
110 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
111 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
112 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
113 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
114 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
115 import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
116 import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
117 import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
118 import org.apache.hadoop.hbase.regionserver.wal.HLog;
119 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
120 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
121 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
122 import org.apache.hadoop.hbase.security.UserProvider;
123 import org.apache.hadoop.hbase.trace.SpanReceiverHost;
124 import org.apache.hadoop.hbase.util.ByteStringer;
125 import org.apache.hadoop.hbase.util.Bytes;
126 import org.apache.hadoop.hbase.util.CompressionTest;
127 import org.apache.hadoop.hbase.util.ConfigUtil;
128 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
129 import org.apache.hadoop.hbase.util.FSTableDescriptors;
130 import org.apache.hadoop.hbase.util.FSUtils;
131 import org.apache.hadoop.hbase.util.HasThread;
132 import org.apache.hadoop.hbase.util.JvmPauseMonitor;
133 import org.apache.hadoop.hbase.util.Sleeper;
134 import org.apache.hadoop.hbase.util.Threads;
135 import org.apache.hadoop.hbase.util.VersionInfo;
136 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
137 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
138 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
139 import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher;
140 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
141 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
142 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
143 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
144 import org.apache.hadoop.ipc.RemoteException;
145 import org.apache.hadoop.metrics.util.MBeanUtil;
146 import org.apache.hadoop.util.ReflectionUtils;
147 import org.apache.hadoop.util.StringUtils;
148 import org.apache.zookeeper.KeeperException;
149 import org.apache.zookeeper.data.Stat;
150 
151 import com.google.common.annotations.VisibleForTesting;
152 import com.google.protobuf.BlockingRpcChannel;
153 import com.google.protobuf.ServiceException;
154 
155 /**
156  * HRegionServer makes a set of HRegions available to clients. It checks in with
157  * the HMaster. There are many HRegionServers in a single HBase deployment.
158  */
159 @InterfaceAudience.Private
160 @SuppressWarnings("deprecation")
161 public class HRegionServer extends HasThread implements
162     RegionServerServices, LastSequenceId {
163 
164   public static final Log LOG = LogFactory.getLog(HRegionServer.class);
165 
166   /*
167    * Strings to be used in forming the exception message for
168    * RegionsAlreadyInTransitionException.
169    */
170   protected static final String OPEN = "OPEN";
171   protected static final String CLOSE = "CLOSE";
172 
173   //RegionName vs current action in progress
174   //true - if open region action in progress
175   //false - if close region action in progress
176   protected final ConcurrentMap<byte[], Boolean> regionsInTransitionInRS =
177     new ConcurrentSkipListMap<byte[], Boolean>(Bytes.BYTES_COMPARATOR);
178 
179   // Cache flushing
180   protected MemStoreFlusher cacheFlusher;
181 
182   protected HeapMemoryManager hMemManager;
183 
184   /*
185    * Short-circuit (ie. bypassing RPC layer) HConnection to this Server
186    * to be used internally for miscellaneous needs. Initialized at the server startup
187    * and closed when server shuts down. Clients must never close it explicitly.
188    */
189   protected HConnection shortCircuitConnection;
190 
191   /*
192    * Long-living meta table locator, which is created when the server is started and stopped
193    * when server shuts down. References to this locator shall be used to perform according
194    * operations in EventHandlers. Primary reason for this decision is to make it mockable
195    * for tests.
196    */
197   protected MetaTableLocator metaTableLocator;
198 
199   // Watch if a region is out of recovering state from ZooKeeper
200   @SuppressWarnings("unused")
201   private RecoveringRegionWatcher recoveringRegionWatcher;
202 
203   /**
204    * Go here to get table descriptors.
205    */
206   protected TableDescriptors tableDescriptors;
207 
208   // Replication services. If no replication, this handler will be null.
209   protected ReplicationSourceService replicationSourceHandler;
210   protected ReplicationSinkService replicationSinkHandler;
211 
212   // Compactions
213   public CompactSplitThread compactSplitThread;
214 
215   /**
216    * Map of regions currently being served by this region server. Key is the
217    * encoded region name.  All access should be synchronized.
218    */
219   protected final Map<String, HRegion> onlineRegions =
220     new ConcurrentHashMap<String, HRegion>();
221 
222   /**
223    * Map of encoded region names to the DataNode locations they should be hosted on
224    * We store the value as InetSocketAddress since this is used only in HDFS
225    * API (create() that takes favored nodes as hints for placing file blocks).
226    * We could have used ServerName here as the value class, but we'd need to
227    * convert it to InetSocketAddress at some point before the HDFS API call, and
228    * it seems a bit weird to store ServerName since ServerName refers to RegionServers
229    * and here we really mean DataNode locations.
230    */
231   protected final Map<String, InetSocketAddress[]> regionFavoredNodesMap =
232       new ConcurrentHashMap<String, InetSocketAddress[]>();
233 
234   /**
235    * Set of regions currently being in recovering state which means it can accept writes(edits from
236    * previous failed region server) but not reads. A recovering region is also an online region.
237    */
238   protected final Map<String, HRegion> recoveringRegions = Collections
239       .synchronizedMap(new HashMap<String, HRegion>());
240 
241   // Leases
242   protected Leases leases;
243 
244   // Instance of the hbase executor service.
245   protected ExecutorService service;
246 
247   // If false, the file system has become unavailable
248   protected volatile boolean fsOk;
249   protected HFileSystem fs;
250 
251   // Set when a report to the master comes back with a message asking us to
252   // shutdown. Also set by call to stop when debugging or running unit tests
253   // of HRegionServer in isolation.
254   private volatile boolean stopped = false;
255 
256   // Go down hard. Used if file system becomes unavailable and also in
257   // debugging and unit tests.
258   private volatile boolean abortRequested;
259 
260   ConcurrentMap<String, Integer> rowlocks = new ConcurrentHashMap<String, Integer>();
261 
262   // A state before we go into stopped state.  At this stage we're closing user
263   // space regions.
264   private boolean stopping = false;
265 
266   private volatile boolean killed = false;
267 
268   protected final Configuration conf;
269 
270   private Path rootDir;
271 
272   protected final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
273 
274   final int numRetries;
275   protected final int threadWakeFrequency;
276   protected final int msgInterval;
277 
278   protected final int numRegionsToReport;
279 
280   // Stub to do region server status calls against the master.
281   private volatile RegionServerStatusService.BlockingInterface rssStub;
282   // RPC client. Used to make the stub above that does region server status checking.
283   RpcClient rpcClient;
284 
285   private UncaughtExceptionHandler uncaughtExceptionHandler;
286 
287   // Info server. Default access so can be used by unit tests. REGIONSERVER
288   // is name of the webapp and the attribute name used stuffing this instance
289   // into web context.
290   protected InfoServer infoServer;
291   private JvmPauseMonitor pauseMonitor;
292 
293   /** region server process name */
294   public static final String REGIONSERVER = "regionserver";
295 
296   MetricsRegionServer metricsRegionServer;
297   private SpanReceiverHost spanReceiverHost;
298 
299   /*
300    * Check for compactions requests.
301    */
302   Chore compactionChecker;
303 
304   /*
305    * Check for flushes
306    */
307   Chore periodicFlusher;
308 
309   // HLog and HLog roller. log is protected rather than private to avoid
310   // eclipse warning when accessed by inner classes
311   protected volatile HLog hlog;
312   // The meta updates are written to a different hlog. If this
313   // regionserver holds meta regions, then this field will be non-null.
314   protected volatile HLog hlogForMeta;
315 
316   LogRoller hlogRoller;
317   LogRoller metaHLogRoller;
318 
319   // flag set after we're done setting up server threads
320   protected AtomicBoolean online;
321 
322   // zookeeper connection and watcher
323   protected ZooKeeperWatcher zooKeeper;
324 
325   // master address tracker
326   private MasterAddressTracker masterAddressTracker;
327 
328   // Cluster Status Tracker
329   protected ClusterStatusTracker clusterStatusTracker;
330 
331   // Log Splitting Worker
332   private SplitLogWorker splitLogWorker;
333 
334   // A sleeper that sleeps for msgInterval.
335   protected final Sleeper sleeper;
336 
337   private final int operationTimeout;
338 
339   private final RegionServerAccounting regionServerAccounting;
340 
341   // Cache configuration and block cache reference
342   final CacheConfig cacheConfig;
343 
344   /** The health check chore. */
345   private HealthCheckChore healthCheckChore;
346 
347   /** The nonce manager chore. */
348   private Chore nonceManagerChore;
349 
350   /**
351    * The server name the Master sees us as.  Its made from the hostname the
352    * master passes us, port, and server startcode. Gets set after registration
353    * against  Master.
354    */
355   protected ServerName serverName;
356 
357   /**
358    * This servers startcode.
359    */
360   protected final long startcode;
361 
362   /**
363    * Unique identifier for the cluster we are a part of.
364    */
365   private String clusterId;
366 
367   /**
368    * MX Bean for RegionServerInfo
369    */
370   private ObjectName mxBean = null;
371 
372   /**
373    * Chore to clean periodically the moved region list
374    */
375   private MovedRegionsCleaner movedRegionsCleaner;
376 
377   // chore for refreshing store files for secondary regions
378   private StorefileRefresherChore storefileRefresher;
379 
380   private RegionServerCoprocessorHost rsHost;
381 
382   private RegionServerProcedureManagerHost rspmHost;
383 
384   // Table level lock manager for locking for region operations
385   protected TableLockManager tableLockManager;
386 
387   /**
388    * Nonce manager. Nonces are used to make operations like increment and append idempotent
389    * in the case where client doesn't receive the response from a successful operation and
390    * retries. We track the successful ops for some time via a nonce sent by client and handle
391    * duplicate operations (currently, by failing them; in future we might use MVCC to return
392    * result). Nonces are also recovered from WAL during, recovery; however, the caveats (from
393    * HBASE-3787) are:
394    * - WAL recovery is optimized, and under high load we won't read nearly nonce-timeout worth
395    *   of past records. If we don't read the records, we don't read and recover the nonces.
396    *   Some WALs within nonce-timeout at recovery may not even be present due to rolling/cleanup.
397    * - There's no WAL recovery during normal region move, so nonces will not be transfered.
398    * We can have separate additional "Nonce WAL". It will just contain bunch of numbers and
399    * won't be flushed on main path - because WAL itself also contains nonces, if we only flush
400    * it before memstore flush, for a given nonce we will either see it in the WAL (if it was
401    * never flushed to disk, it will be part of recovery), or we'll see it as part of the nonce
402    * log (or both occasionally, which doesn't matter). Nonce log file can be deleted after the
403    * latest nonce in it expired. It can also be recovered during move.
404    */
405   final ServerNonceManager nonceManager;
406 
407   private UserProvider userProvider;
408 
409   protected final RSRpcServices rpcServices;
410 
411   protected BaseCoordinatedStateManager csm;
412 
413   private final boolean useZKForAssignment;
414 
415   /**
416    * Starts a HRegionServer at the default location.
417    * @param conf
418    * @throws IOException
419    * @throws InterruptedException
420    */
421   public HRegionServer(Configuration conf) throws IOException, InterruptedException {
422     this(conf, CoordinatedStateManagerFactory.getCoordinatedStateManager(conf));
423   }
424 
425   /**
426    * Starts a HRegionServer at the default location
427    * @param conf
428    * @param csm implementation of CoordinatedStateManager to be used
429    * @throws IOException
430    * @throws InterruptedException
431    */
432   public HRegionServer(Configuration conf, CoordinatedStateManager csm)
433       throws IOException, InterruptedException {
434     this.fsOk = true;
435     this.conf = conf;
436     checkCodecs(this.conf);
437     this.online = new AtomicBoolean(false);
438     this.userProvider = UserProvider.instantiate(conf);
439     FSUtils.setupShortCircuitRead(this.conf);
440 
441     // Config'ed params
442     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
443         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
444     this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
445     this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
446 
447     this.sleeper = new Sleeper(this.msgInterval, this);
448 
449     boolean isNoncesEnabled = conf.getBoolean(HConstants.HBASE_RS_NONCES_ENABLED, true);
450     this.nonceManager = isNoncesEnabled ? new ServerNonceManager(this.conf) : null;
451 
452     this.numRegionsToReport = conf.getInt(
453       "hbase.regionserver.numregionstoreport", 10);
454 
455     this.operationTimeout = conf.getInt(
456       HConstants.HBASE_RPC_SHORTOPERATION_TIMEOUT_KEY,
457       HConstants.DEFAULT_HBASE_RPC_SHORTOPERATION_TIMEOUT);
458 
459     this.abortRequested = false;
460     this.stopped = false;
461 
462     rpcServices = createRpcServices();
463     this.startcode = System.currentTimeMillis();
464     String hostName = rpcServices.isa.getHostName();
465     serverName = ServerName.valueOf(hostName, rpcServices.isa.getPort(), startcode);
466 
467     // login the zookeeper client principal (if using security)
468     ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
469       "hbase.zookeeper.client.kerberos.principal", hostName);
470     // login the server principal (if using secure Hadoop)
471     login(userProvider, hostName);
472 
473     regionServerAccounting = new RegionServerAccounting();
474     cacheConfig = new CacheConfig(conf);
475     uncaughtExceptionHandler = new UncaughtExceptionHandler() {
476       @Override
477       public void uncaughtException(Thread t, Throwable e) {
478         abort("Uncaught exception in service thread " + t.getName(), e);
479       }
480     };
481 
482     useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
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 this.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         LOG.error("Metalog close and delete failed", RemoteExceptionHandler.checkThrowable(e));
1116       }
1117     }
1118     if (this.hlog != null) {
1119       try {
1120         if (delete) {
1121           hlog.closeAndDelete();
1122         } else {
1123           hlog.close();
1124         }
1125       } catch (Throwable e) {
1126         LOG.error("Close and delete failed", RemoteExceptionHandler.checkThrowable(e));
1127       }
1128     }
1129   }
1130 
1131   /*
1132    * Run init. Sets up hlog and starts up all server threads.
1133    *
1134    * @param c Extra configuration.
1135    */
1136   protected void handleReportForDutyResponse(final RegionServerStartupResponse c)
1137   throws IOException {
1138     try {
1139       for (NameStringPair e : c.getMapEntriesList()) {
1140         String key = e.getName();
1141         // The hostname the master sees us as.
1142         if (key.equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
1143           String hostnameFromMasterPOV = e.getValue();
1144           this.serverName = ServerName.valueOf(hostnameFromMasterPOV,
1145             rpcServices.isa.getPort(), this.startcode);
1146           if (!hostnameFromMasterPOV.equals(rpcServices.isa.getHostName())) {
1147             LOG.info("Master passed us a different hostname to use; was=" +
1148               rpcServices.isa.getHostName() + ", but now=" + hostnameFromMasterPOV);
1149           }
1150           continue;
1151         }
1152         String value = e.getValue();
1153         if (LOG.isDebugEnabled()) {
1154           LOG.info("Config from master: " + key + "=" + value);
1155         }
1156         this.conf.set(key, value);
1157       }
1158 
1159       // hack! Maps DFSClient => RegionServer for logs.  HDFS made this
1160       // config param for task trackers, but we can piggyback off of it.
1161       if (this.conf.get("mapreduce.task.attempt.id") == null) {
1162         this.conf.set("mapreduce.task.attempt.id", "hb_rs_" +
1163           this.serverName.toString());
1164       }
1165 
1166       // Save it in a file, this will allow to see if we crash
1167       ZNodeClearer.writeMyEphemeralNodeOnDisk(getMyEphemeralNodePath());
1168 
1169       this.hlog = setupWALAndReplication();
1170       // Init in here rather than in constructor after thread name has been set
1171       this.metricsRegionServer = new MetricsRegionServer(new MetricsRegionServerWrapperImpl(this));
1172 
1173       startServiceThreads();
1174       startHeapMemoryManager();
1175       LOG.info("Serving as " + this.serverName +
1176         ", RpcServer on " + rpcServices.isa +
1177         ", sessionid=0x" +
1178         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
1179       synchronized (online) {
1180         online.set(true);
1181         online.notifyAll();
1182       }
1183     } catch (Throwable e) {
1184       stop("Failed initialization");
1185       throw convertThrowableToIOE(cleanup(e, "Failed init"),
1186           "Region server startup failed");
1187     } finally {
1188       sleeper.skipSleepCycle();
1189     }
1190   }
1191 
1192   private void startHeapMemoryManager() {
1193     this.hMemManager = HeapMemoryManager.create(this.conf, this.cacheFlusher, this);
1194     if (this.hMemManager != null) {
1195       this.hMemManager.start();
1196     }
1197   }
1198 
1199   private void createMyEphemeralNode() throws KeeperException, IOException {
1200     RegionServerInfo.Builder rsInfo = RegionServerInfo.newBuilder();
1201     rsInfo.setInfoPort(infoServer != null ? infoServer.getPort() : -1);
1202     byte[] data = ProtobufUtil.prependPBMagic(rsInfo.build().toByteArray());
1203     ZKUtil.createEphemeralNodeAndWatch(this.zooKeeper,
1204       getMyEphemeralNodePath(), data);
1205   }
1206 
1207   private void deleteMyEphemeralNode() throws KeeperException {
1208     ZKUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath());
1209   }
1210 
1211   @Override
1212   public RegionServerAccounting getRegionServerAccounting() {
1213     return regionServerAccounting;
1214   }
1215 
1216   @Override
1217   public TableLockManager getTableLockManager() {
1218     return tableLockManager;
1219   }
1220 
1221   /*
1222    * @param r Region to get RegionLoad for.
1223    * @param regionLoadBldr the RegionLoad.Builder, can be null
1224    * @param regionSpecifier the RegionSpecifier.Builder, can be null
1225    * @return RegionLoad instance.
1226    *
1227    * @throws IOException
1228    */
1229   private RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr,
1230       RegionSpecifier.Builder regionSpecifier) {
1231     byte[] name = r.getRegionName();
1232     int stores = 0;
1233     int storefiles = 0;
1234     int storeUncompressedSizeMB = 0;
1235     int storefileSizeMB = 0;
1236     int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
1237     int storefileIndexSizeMB = 0;
1238     int rootIndexSizeKB = 0;
1239     int totalStaticIndexSizeKB = 0;
1240     int totalStaticBloomSizeKB = 0;
1241     long totalCompactingKVs = 0;
1242     long currentCompactedKVs = 0;
1243     synchronized (r.stores) {
1244       stores += r.stores.size();
1245       for (Store store : r.stores.values()) {
1246         storefiles += store.getStorefilesCount();
1247         storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed()
1248             / 1024 / 1024);
1249         storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
1250         storefileIndexSizeMB += (int) (store.getStorefilesIndexSize() / 1024 / 1024);
1251         CompactionProgress progress = store.getCompactionProgress();
1252         if (progress != null) {
1253           totalCompactingKVs += progress.totalCompactingKVs;
1254           currentCompactedKVs += progress.currentCompactedKVs;
1255         }
1256 
1257         rootIndexSizeKB +=
1258             (int) (store.getStorefilesIndexSize() / 1024);
1259 
1260         totalStaticIndexSizeKB +=
1261           (int) (store.getTotalStaticIndexSize() / 1024);
1262 
1263         totalStaticBloomSizeKB +=
1264           (int) (store.getTotalStaticBloomSize() / 1024);
1265       }
1266     }
1267     if (regionLoadBldr == null) {
1268       regionLoadBldr = RegionLoad.newBuilder();
1269     }
1270     if (regionSpecifier == null) {
1271       regionSpecifier = RegionSpecifier.newBuilder();
1272     }
1273     regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
1274     regionSpecifier.setValue(ByteStringer.wrap(name));
1275     regionLoadBldr.setRegionSpecifier(regionSpecifier.build())
1276       .setStores(stores)
1277       .setStorefiles(storefiles)
1278       .setStoreUncompressedSizeMB(storeUncompressedSizeMB)
1279       .setStorefileSizeMB(storefileSizeMB)
1280       .setMemstoreSizeMB(memstoreSizeMB)
1281       .setStorefileIndexSizeMB(storefileIndexSizeMB)
1282       .setRootIndexSizeKB(rootIndexSizeKB)
1283       .setTotalStaticIndexSizeKB(totalStaticIndexSizeKB)
1284       .setTotalStaticBloomSizeKB(totalStaticBloomSizeKB)
1285       .setReadRequestsCount(r.readRequestsCount.get())
1286       .setWriteRequestsCount(r.writeRequestsCount.get())
1287       .setTotalCompactingKVs(totalCompactingKVs)
1288       .setCurrentCompactedKVs(currentCompactedKVs)
1289       .setCompleteSequenceId(r.lastFlushSeqId);
1290 
1291     return regionLoadBldr.build();
1292   }
1293 
1294   /**
1295    * @param encodedRegionName
1296    * @return An instance of RegionLoad.
1297    */
1298   public RegionLoad createRegionLoad(final String encodedRegionName) {
1299     HRegion r = null;
1300     r = this.onlineRegions.get(encodedRegionName);
1301     return r != null ? createRegionLoad(r, null, null) : null;
1302   }
1303 
1304   /*
1305    * Inner class that runs on a long period checking if regions need compaction.
1306    */
1307   private static class CompactionChecker extends Chore {
1308     private final HRegionServer instance;
1309     private final int majorCompactPriority;
1310     private final static int DEFAULT_PRIORITY = Integer.MAX_VALUE;
1311     private long iteration = 0;
1312 
1313     CompactionChecker(final HRegionServer h, final int sleepTime,
1314         final Stoppable stopper) {
1315       super("CompactionChecker", sleepTime, h);
1316       this.instance = h;
1317       LOG.info(this.getName() + " runs every " + StringUtils.formatTime(sleepTime));
1318 
1319       /* MajorCompactPriority is configurable.
1320        * If not set, the compaction will use default priority.
1321        */
1322       this.majorCompactPriority = this.instance.conf.
1323         getInt("hbase.regionserver.compactionChecker.majorCompactPriority",
1324         DEFAULT_PRIORITY);
1325     }
1326 
1327     @Override
1328     protected void chore() {
1329       for (HRegion r : this.instance.onlineRegions.values()) {
1330         if (r == null)
1331           continue;
1332         for (Store s : r.getStores().values()) {
1333           try {
1334             long multiplier = s.getCompactionCheckMultiplier();
1335             assert multiplier > 0;
1336             if (iteration % multiplier != 0) continue;
1337             if (s.needsCompaction()) {
1338               // Queue a compaction. Will recognize if major is needed.
1339               this.instance.compactSplitThread.requestSystemCompaction(r, s, getName()
1340                   + " requests compaction");
1341             } else if (s.isMajorCompaction()) {
1342               if (majorCompactPriority == DEFAULT_PRIORITY
1343                   || majorCompactPriority > r.getCompactPriority()) {
1344                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1345                     + " requests major compaction; use default priority", null);
1346               } else {
1347                 this.instance.compactSplitThread.requestCompaction(r, s, getName()
1348                     + " requests major compaction; use configured priority",
1349                   this.majorCompactPriority, null);
1350               }
1351             }
1352           } catch (IOException e) {
1353             LOG.warn("Failed major compaction check on " + r, e);
1354           }
1355         }
1356       }
1357       iteration = (iteration == Long.MAX_VALUE) ? 0 : (iteration + 1);
1358     }
1359   }
1360 
1361   class PeriodicMemstoreFlusher extends Chore {
1362     final HRegionServer server;
1363     final static int RANGE_OF_DELAY = 20000; //millisec
1364     final static int MIN_DELAY_TIME = 3000; //millisec
1365     public PeriodicMemstoreFlusher(int cacheFlushInterval, final HRegionServer server) {
1366       super(server.getServerName() + "-MemstoreFlusherChore", cacheFlushInterval, server);
1367       this.server = server;
1368     }
1369 
1370     @Override
1371     protected void chore() {
1372       for (HRegion r : this.server.onlineRegions.values()) {
1373         if (r == null)
1374           continue;
1375         if (r.shouldFlush()) {
1376           FlushRequester requester = server.getFlushRequester();
1377           if (requester != null) {
1378             long randomDelay = RandomUtils.nextInt(RANGE_OF_DELAY) + MIN_DELAY_TIME;
1379             LOG.info(getName() + " requesting flush for region " + r.getRegionNameAsString() +
1380                 " after a delay of " + randomDelay);
1381             //Throttle the flushes by putting a delay. If we don't throttle, and there
1382             //is a balanced write-load on the regions in a table, we might end up
1383             //overwhelming the filesystem with too many flushes at once.
1384             requester.requestDelayedFlush(r, randomDelay);
1385           }
1386         }
1387       }
1388     }
1389   }
1390 
1391   /**
1392    * Report the status of the server. A server is online once all the startup is
1393    * completed (setting up filesystem, starting service threads, etc.). This
1394    * method is designed mostly to be useful in tests.
1395    *
1396    * @return true if online, false if not.
1397    */
1398   public boolean isOnline() {
1399     return online.get();
1400   }
1401 
1402   /**
1403    * Setup WAL log and replication if enabled.
1404    * Replication setup is done in here because it wants to be hooked up to WAL.
1405    * @return A WAL instance.
1406    * @throws IOException
1407    */
1408   private HLog setupWALAndReplication() throws IOException {
1409     final Path oldLogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1410     final String logName
1411       = HLogUtil.getHLogDirectoryName(this.serverName.toString());
1412 
1413     Path logdir = new Path(rootDir, logName);
1414     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1415     if (this.fs.exists(logdir)) {
1416       throw new RegionServerRunningException("Region server has already " +
1417         "created directory at " + this.serverName.toString());
1418     }
1419 
1420     // Instantiate replication manager if replication enabled.  Pass it the
1421     // log directories.
1422     createNewReplicationInstance(conf, this, this.fs, logdir, oldLogDir);
1423 
1424     return instantiateHLog(rootDir, logName);
1425   }
1426 
1427   private HLog getMetaWAL() throws IOException {
1428     if (this.hlogForMeta != null) return this.hlogForMeta;
1429     final String logName = HLogUtil.getHLogDirectoryName(this.serverName.toString());
1430     Path logdir = new Path(rootDir, logName);
1431     if (LOG.isDebugEnabled()) LOG.debug("logdir=" + logdir);
1432     this.hlogForMeta = HLogFactory.createMetaHLog(this.fs.getBackingFs(), rootDir, logName,
1433       this.conf, getMetaWALActionListeners(), this.serverName.toString());
1434     return this.hlogForMeta;
1435   }
1436 
1437   /**
1438    * Called by {@link #setupWALAndReplication()} creating WAL instance.
1439    * @param rootdir
1440    * @param logName
1441    * @return WAL instance.
1442    * @throws IOException
1443    */
1444   protected HLog instantiateHLog(Path rootdir, String logName) throws IOException {
1445     return HLogFactory.createHLog(this.fs.getBackingFs(), rootdir, logName, this.conf,
1446       getWALActionListeners(), this.serverName.toString());
1447   }
1448 
1449   /**
1450    * Called by {@link #instantiateHLog(Path, String)} setting up WAL instance.
1451    * Add any {@link WALActionsListener}s you want inserted before WAL startup.
1452    * @return List of WALActionsListener that will be passed in to
1453    * {@link org.apache.hadoop.hbase.regionserver.wal.FSHLog} on construction.
1454    */
1455   protected List<WALActionsListener> getWALActionListeners() {
1456     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1457     // Log roller.
1458     this.hlogRoller = new LogRoller(this, this);
1459     listeners.add(this.hlogRoller);
1460     if (this.replicationSourceHandler != null &&
1461         this.replicationSourceHandler.getWALActionsListener() != null) {
1462       // Replication handler is an implementation of WALActionsListener.
1463       listeners.add(this.replicationSourceHandler.getWALActionsListener());
1464     }
1465     return listeners;
1466   }
1467 
1468   protected List<WALActionsListener> getMetaWALActionListeners() {
1469     List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
1470     // Using a tmp log roller to ensure metaLogRoller is alive once it is not
1471     // null
1472     MetaLogRoller tmpLogRoller = new MetaLogRoller(this, this);
1473     String n = Thread.currentThread().getName();
1474     Threads.setDaemonThreadRunning(tmpLogRoller.getThread(),
1475         n + "-MetaLogRoller", uncaughtExceptionHandler);
1476     this.metaHLogRoller = tmpLogRoller;
1477     tmpLogRoller = null;
1478     listeners.add(this.metaHLogRoller);
1479     return listeners;
1480   }
1481 
1482   protected LogRoller getLogRoller() {
1483     return hlogRoller;
1484   }
1485 
1486   public MetricsRegionServer getRegionServerMetrics() {
1487     return this.metricsRegionServer;
1488   }
1489 
1490   /**
1491    * @return Master address tracker instance.
1492    */
1493   public MasterAddressTracker getMasterAddressTracker() {
1494     return this.masterAddressTracker;
1495   }
1496 
1497   /*
1498    * Start maintenance Threads, Server, Worker and lease checker threads.
1499    * Install an UncaughtExceptionHandler that calls abort of RegionServer if we
1500    * get an unhandled exception. We cannot set the handler on all threads.
1501    * Server's internal Listener thread is off limits. For Server, if an OOME, it
1502    * waits a while then retries. Meantime, a flush or a compaction that tries to
1503    * run should trigger same critical condition and the shutdown will run. On
1504    * its way out, this server will shut down Server. Leases are sort of
1505    * inbetween. It has an internal thread that while it inherits from Chore, it
1506    * keeps its own internal stop mechanism so needs to be stopped by this
1507    * hosting server. Worker logs the exception and exits.
1508    */
1509   private void startServiceThreads() throws IOException {
1510     // Start executor services
1511     this.service.startExecutorService(ExecutorType.RS_OPEN_REGION,
1512       conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1513     this.service.startExecutorService(ExecutorType.RS_OPEN_META,
1514       conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1515     this.service.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1516       conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1517     this.service.startExecutorService(ExecutorType.RS_CLOSE_META,
1518       conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1519     if (conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false)) {
1520       this.service.startExecutorService(ExecutorType.RS_PARALLEL_SEEK,
1521         conf.getInt("hbase.storescanner.parallel.seek.threads", 10));
1522     }
1523     this.service.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS,
1524       conf.getInt("hbase.regionserver.wal.max.splitters", SplitLogWorker.DEFAULT_MAX_SPLITTERS));
1525 
1526     Threads.setDaemonThreadRunning(this.hlogRoller.getThread(), getName() + ".logRoller",
1527         uncaughtExceptionHandler);
1528     this.cacheFlusher.start(uncaughtExceptionHandler);
1529     Threads.setDaemonThreadRunning(this.compactionChecker.getThread(), getName() +
1530       ".compactionChecker", uncaughtExceptionHandler);
1531     Threads.setDaemonThreadRunning(this.periodicFlusher.getThread(), getName() +
1532         ".periodicFlusher", uncaughtExceptionHandler);
1533     if (this.healthCheckChore != null) {
1534       Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), getName() + ".healthChecker",
1535             uncaughtExceptionHandler);
1536     }
1537     if (this.nonceManagerChore != null) {
1538       Threads.setDaemonThreadRunning(this.nonceManagerChore.getThread(), getName() + ".nonceCleaner",
1539             uncaughtExceptionHandler);
1540     }
1541     if (this.storefileRefresher != null) {
1542       Threads.setDaemonThreadRunning(this.storefileRefresher.getThread(), getName() + ".storefileRefresher",
1543             uncaughtExceptionHandler);
1544     }
1545 
1546     // Leases is not a Thread. Internally it runs a daemon thread. If it gets
1547     // an unhandled exception, it will just exit.
1548     this.leases.setName(getName() + ".leaseChecker");
1549     this.leases.start();
1550 
1551     if (this.replicationSourceHandler == this.replicationSinkHandler &&
1552         this.replicationSourceHandler != null) {
1553       this.replicationSourceHandler.startReplicationService();
1554     } else {
1555       if (this.replicationSourceHandler != null) {
1556         this.replicationSourceHandler.startReplicationService();
1557       }
1558       if (this.replicationSinkHandler != null) {
1559         this.replicationSinkHandler.startReplicationService();
1560       }
1561     }
1562 
1563     // Start Server.  This service is like leases in that it internally runs
1564     // a thread.
1565     rpcServices.rpcServer.start();
1566 
1567     // Create the log splitting worker and start it
1568     // set a smaller retries to fast fail otherwise splitlogworker could be blocked for
1569     // quite a while inside HConnection layer. The worker won't be available for other
1570     // tasks even after current task is preempted after a split task times out.
1571     Configuration sinkConf = HBaseConfiguration.create(conf);
1572     sinkConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
1573       conf.getInt("hbase.log.replay.retries.number", 8)); // 8 retries take about 23 seconds
1574     sinkConf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
1575       conf.getInt("hbase.log.replay.rpc.timeout", 30000)); // default 30 seconds
1576     sinkConf.setInt("hbase.client.serverside.retries.multiplier", 1);
1577     this.splitLogWorker = new SplitLogWorker(this.zooKeeper, sinkConf, this, this);
1578     splitLogWorker.start();
1579   }
1580 
1581   /**
1582    * Puts up the webui.
1583    * @return Returns final port -- maybe different from what we started with.
1584    * @throws IOException
1585    */
1586   private int putUpWebUI() throws IOException {
1587     int port = this.conf.getInt(HConstants.REGIONSERVER_INFO_PORT,
1588       HConstants.DEFAULT_REGIONSERVER_INFOPORT);
1589     // -1 is for disabling info server
1590     if (port < 0) return port;
1591     String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
1592     // check if auto port bind enabled
1593     boolean auto = this.conf.getBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO,
1594         false);
1595     while (true) {
1596       try {
1597         this.infoServer = new InfoServer(getProcessName(), addr, port, false, this.conf);
1598         infoServer.addServlet("dump", "/dump", getDumpServlet());
1599         configureInfoServer();
1600         this.infoServer.start();
1601         break;
1602       } catch (BindException e) {
1603         if (!auto) {
1604           // auto bind disabled throw BindException
1605           LOG.error("Failed binding http info server to port: " + port);
1606           throw e;
1607         }
1608         // auto bind enabled, try to use another port
1609         LOG.info("Failed binding http info server to port: " + port);
1610         port++;
1611       }
1612     }
1613     port = this.infoServer.getPort();
1614     conf.setInt(HConstants.REGIONSERVER_INFO_PORT, port);
1615     int masterInfoPort = conf.getInt(HConstants.MASTER_INFO_PORT,
1616       HConstants.DEFAULT_MASTER_INFOPORT);
1617     conf.setInt("hbase.master.info.port.orig", masterInfoPort);
1618     conf.setInt(HConstants.MASTER_INFO_PORT, port);
1619     return port;
1620   }
1621 
1622   /*
1623    * Verify that server is healthy
1624    */
1625   private boolean isHealthy() {
1626     if (!fsOk) {
1627       // File system problem
1628       return false;
1629     }
1630     // Verify that all threads are alive
1631     if (!(leases.isAlive()
1632         && cacheFlusher.isAlive() && hlogRoller.isAlive()
1633         && this.compactionChecker.isAlive()
1634         && this.periodicFlusher.isAlive())) {
1635       stop("One or more threads are no longer alive -- stop");
1636       return false;
1637     }
1638     if (metaHLogRoller != null && !metaHLogRoller.isAlive()) {
1639       stop("Meta HLog roller thread is no longer alive -- stop");
1640       return false;
1641     }
1642     return true;
1643   }
1644 
1645   public HLog getWAL() {
1646     try {
1647       return getWAL(null);
1648     } catch (IOException e) {
1649       LOG.warn("getWAL threw exception " + e);
1650       return null;
1651     }
1652   }
1653 
1654   @Override
1655   public HLog getWAL(HRegionInfo regionInfo) throws IOException {
1656     //TODO: at some point this should delegate to the HLogFactory
1657     //currently, we don't care about the region as much as we care about the
1658     //table.. (hence checking the tablename below)
1659     //_ROOT_ and hbase:meta regions have separate WAL.
1660     if (regionInfo != null && regionInfo.isMetaTable()) {
1661       return getMetaWAL();
1662     }
1663     return this.hlog;
1664   }
1665 
1666   @Override
1667   public HConnection getShortCircuitConnection() {
1668     return this.shortCircuitConnection;
1669   }
1670 
1671   @Override
1672   public MetaTableLocator getMetaTableLocator() {
1673     return this.metaTableLocator;
1674   }
1675 
1676   @Override
1677   public void stop(final String msg) {
1678     if (!this.stopped) {
1679       try {
1680         if (this.rsHost != null) {
1681           this.rsHost.preStop(msg);
1682         }
1683         this.stopped = true;
1684         LOG.info("STOPPED: " + msg);
1685         // Wakes run() if it is sleeping
1686         sleeper.skipSleepCycle();
1687       } catch (IOException exp) {
1688         LOG.warn("The region server did not stop", exp);
1689       }
1690     }
1691   }
1692 
1693   public void waitForServerOnline(){
1694     while (!isOnline() && !isStopped()){
1695        sleeper.sleep();
1696     }
1697   }
1698 
1699   @Override
1700   public void postOpenDeployTasks(final HRegion r)
1701   throws KeeperException, IOException {
1702     rpcServices.checkOpen();
1703     LOG.info("Post open deploy tasks for " + r.getRegionNameAsString());
1704     // Do checks to see if we need to compact (references or too many files)
1705     for (Store s : r.getStores().values()) {
1706       if (s.hasReferences() || s.needsCompaction()) {
1707        this.compactSplitThread.requestSystemCompaction(r, s, "Opening Region");
1708       }
1709     }
1710     long openSeqNum = r.getOpenSeqNum();
1711     if (openSeqNum == HConstants.NO_SEQNUM) {
1712       // If we opened a region, we should have read some sequence number from it.
1713       LOG.error("No sequence number found when opening " + r.getRegionNameAsString());
1714       openSeqNum = 0;
1715     }
1716 
1717     // Update flushed sequence id of a recovering region in ZK
1718     updateRecoveringRegionLastFlushedSequenceId(r);
1719 
1720     // Update ZK, or META
1721     if (r.getRegionInfo().isMetaRegion()) {
1722       MetaTableLocator.setMetaLocation(getZooKeeper(), serverName);
1723     } else if (useZKForAssignment) {
1724       MetaTableAccessor.updateRegionLocation(shortCircuitConnection, r.getRegionInfo(),
1725         this.serverName, openSeqNum);
1726     }
1727     if (!useZKForAssignment && !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     ReportRegionStateTransitionRequest.Builder builder =
1745       ReportRegionStateTransitionRequest.newBuilder();
1746     builder.setServer(ProtobufUtil.toServerName(serverName));
1747     RegionStateTransition.Builder transition = builder.addTransitionBuilder();
1748     transition.setTransitionCode(code);
1749     if (code == TransitionCode.OPENED && openSeqNum >= 0) {
1750       transition.setOpenSeqNum(openSeqNum);
1751     }
1752     for (HRegionInfo hri: hris) {
1753       transition.addRegionInfo(HRegionInfo.convert(hri));
1754     }
1755     ReportRegionStateTransitionRequest request = builder.build();
1756     while (keepLooping()) {
1757       RegionServerStatusService.BlockingInterface rss = rssStub;
1758       try {
1759         if (rss == null) {
1760           createRegionServerStatusStub();
1761           continue;
1762         }
1763         ReportRegionStateTransitionResponse response =
1764           rss.reportRegionStateTransition(null, request);
1765         if (response.hasErrorMessage()) {
1766           LOG.info("Failed to transition " + hris[0]
1767             + " to " + code + ": " + response.getErrorMessage());
1768           return false;
1769         }
1770         return true;
1771       } catch (ServiceException se) {
1772         IOException ioe = ProtobufUtil.getRemoteException(se);
1773         LOG.info("Failed to report region transition, will retry", ioe);
1774         if (rssStub == rss) {
1775           rssStub = null;
1776         }
1777       }
1778     }
1779     return false;
1780   }
1781 
1782   @Override
1783   public RpcServerInterface getRpcServer() {
1784     return rpcServices.rpcServer;
1785   }
1786 
1787   @VisibleForTesting
1788   public RSRpcServices getRSRpcServices() {
1789     return rpcServices;
1790   }
1791 
1792   /**
1793    * Cause the server to exit without closing the regions it is serving, the log
1794    * it is using and without notifying the master. Used unit testing and on
1795    * catastrophic events such as HDFS is yanked out from under hbase or we OOME.
1796    *
1797    * @param reason
1798    *          the reason we are aborting
1799    * @param cause
1800    *          the exception that caused the abort, or null
1801    */
1802   @Override
1803   public void abort(String reason, Throwable cause) {
1804     String msg = "ABORTING region server " + this + ": " + reason;
1805     if (cause != null) {
1806       LOG.fatal(msg, cause);
1807     } else {
1808       LOG.fatal(msg);
1809     }
1810     this.abortRequested = true;
1811     // HBASE-4014: show list of coprocessors that were loaded to help debug
1812     // regionserver crashes.Note that we're implicitly using
1813     // java.util.HashSet's toString() method to print the coprocessor names.
1814     LOG.fatal("RegionServer abort: loaded coprocessors are: " +
1815         CoprocessorHost.getLoadedCoprocessors());
1816     // Do our best to report our abort to the master, but this may not work
1817     try {
1818       if (cause != null) {
1819         msg += "\nCause:\n" + StringUtils.stringifyException(cause);
1820       }
1821       // Report to the master but only if we have already registered with the master.
1822       if (rssStub != null && this.serverName != null) {
1823         ReportRSFatalErrorRequest.Builder builder =
1824           ReportRSFatalErrorRequest.newBuilder();
1825         ServerName sn =
1826           ServerName.parseVersionedServerName(this.serverName.getVersionedBytes());
1827         builder.setServer(ProtobufUtil.toServerName(sn));
1828         builder.setErrorMessage(msg);
1829         rssStub.reportRSFatalError(null, builder.build());
1830       }
1831     } catch (Throwable t) {
1832       LOG.warn("Unable to report fatal error to master", t);
1833     }
1834     stop(reason);
1835   }
1836 
1837   /**
1838    * @see HRegionServer#abort(String, Throwable)
1839    */
1840   public void abort(String reason) {
1841     abort(reason, null);
1842   }
1843 
1844   @Override
1845   public boolean isAborted() {
1846     return this.abortRequested;
1847   }
1848 
1849   /*
1850    * Simulate a kill -9 of this server. Exits w/o closing regions or cleaninup
1851    * logs but it does close socket in case want to bring up server on old
1852    * hostname+port immediately.
1853    */
1854   protected void kill() {
1855     this.killed = true;
1856     abort("Simulated kill");
1857   }
1858 
1859   /**
1860    * Wait on all threads to finish. Presumption is that all closes and stops
1861    * have already been called.
1862    */
1863   protected void stopServiceThreads() {
1864     if (this.nonceManagerChore != null) {
1865       Threads.shutdown(this.nonceManagerChore.getThread());
1866     }
1867     if (this.compactionChecker != null) {
1868       Threads.shutdown(this.compactionChecker.getThread());
1869     }
1870     if (this.periodicFlusher != null) {
1871       Threads.shutdown(this.periodicFlusher.getThread());
1872     }
1873     if (this.cacheFlusher != null) {
1874       this.cacheFlusher.join();
1875     }
1876     if (this.healthCheckChore != null) {
1877       Threads.shutdown(this.healthCheckChore.getThread());
1878     }
1879     if (this.spanReceiverHost != null) {
1880       this.spanReceiverHost.closeReceivers();
1881     }
1882     if (this.hlogRoller != null) {
1883       Threads.shutdown(this.hlogRoller.getThread());
1884     }
1885     if (this.metaHLogRoller != null) {
1886       Threads.shutdown(this.metaHLogRoller.getThread());
1887     }
1888     if (this.compactSplitThread != null) {
1889       this.compactSplitThread.join();
1890     }
1891     if (this.service != null) this.service.shutdown();
1892     if (this.replicationSourceHandler != null &&
1893         this.replicationSourceHandler == this.replicationSinkHandler) {
1894       this.replicationSourceHandler.stopReplicationService();
1895     } else {
1896       if (this.replicationSourceHandler != null) {
1897         this.replicationSourceHandler.stopReplicationService();
1898       }
1899       if (this.replicationSinkHandler != null) {
1900         this.replicationSinkHandler.stopReplicationService();
1901       }
1902     }
1903     if (this.storefileRefresher != null) {
1904       Threads.shutdown(this.storefileRefresher.getThread());
1905     }
1906   }
1907 
1908   /**
1909    * @return Return the object that implements the replication
1910    * source service.
1911    */
1912   ReplicationSourceService getReplicationSourceService() {
1913     return replicationSourceHandler;
1914   }
1915 
1916   /**
1917    * @return Return the object that implements the replication
1918    * sink service.
1919    */
1920   ReplicationSinkService getReplicationSinkService() {
1921     return replicationSinkHandler;
1922   }
1923 
1924   /**
1925    * Get the current master from ZooKeeper and open the RPC connection to it.
1926    *
1927    * Method will block until a master is available. You can break from this
1928    * block by requesting the server stop.
1929    *
1930    * @return master + port, or null if server has been stopped
1931    */
1932   private synchronized ServerName createRegionServerStatusStub() {
1933     if (rssStub != null) {
1934       return masterAddressTracker.getMasterAddress();
1935     }
1936     ServerName sn = null;
1937     long previousLogTime = 0;
1938     RegionServerStatusService.BlockingInterface master = null;
1939     boolean refresh = false; // for the first time, use cached data
1940     RegionServerStatusService.BlockingInterface intf = null;
1941     boolean interrupted = false;
1942     try {
1943       while (keepLooping() && master == null) {
1944         sn = this.masterAddressTracker.getMasterAddress(refresh);
1945         if (sn == null) {
1946           if (!keepLooping()) {
1947             // give up with no connection.
1948             LOG.debug("No master found and cluster is stopped; bailing out");
1949             return null;
1950           }
1951           LOG.debug("No master found; retry");
1952           previousLogTime = System.currentTimeMillis();
1953           refresh = true; // let's try pull it from ZK directly
1954           sleeper.sleep();
1955           continue;
1956         }
1957 
1958         // If we are on the active master, use the shortcut
1959         if (this instanceof HMaster && sn.equals(getServerName())) {
1960           intf = ((HMaster)this).getMasterRpcServices();
1961           break;
1962         }
1963         try {
1964           BlockingRpcChannel channel =
1965             this.rpcClient.createBlockingRpcChannel(sn, userProvider.getCurrent(), operationTimeout);
1966           intf = RegionServerStatusService.newBlockingStub(channel);
1967           break;
1968         } catch (IOException e) {
1969           e = e instanceof RemoteException ?
1970             ((RemoteException)e).unwrapRemoteException() : e;
1971           if (e instanceof ServerNotRunningYetException) {
1972             if (System.currentTimeMillis() > (previousLogTime+1000)){
1973               LOG.info("Master isn't available yet, retrying");
1974               previousLogTime = System.currentTimeMillis();
1975             }
1976           } else {
1977             if (System.currentTimeMillis() > (previousLogTime + 1000)) {
1978               LOG.warn("Unable to connect to master. Retrying. Error was:", e);
1979               previousLogTime = System.currentTimeMillis();
1980             }
1981           }
1982           try {
1983             Thread.sleep(200);
1984           } catch (InterruptedException ex) {
1985             interrupted = true;
1986             LOG.warn("Interrupted while sleeping");
1987           }
1988         }
1989       }
1990     } finally {
1991       if (interrupted) {
1992         Thread.currentThread().interrupt();
1993       }
1994     }
1995     rssStub = intf;
1996     return sn;
1997   }
1998 
1999   /**
2000    * @return True if we should break loop because cluster is going down or
2001    * this server has been stopped or hdfs has gone bad.
2002    */
2003   private boolean keepLooping() {
2004     return !this.stopped && isClusterUp();
2005   }
2006 
2007   /*
2008    * Let the master know we're here Run initialization using parameters passed
2009    * us by the master.
2010    * @return A Map of key/value configurations we got from the Master else
2011    * null if we failed to register.
2012    * @throws IOException
2013    */
2014   private RegionServerStartupResponse reportForDuty() throws IOException {
2015     ServerName masterServerName = createRegionServerStatusStub();
2016     if (masterServerName == null) return null;
2017     RegionServerStartupResponse result = null;
2018     try {
2019       rpcServices.requestCount.set(0);
2020       LOG.info("reportForDuty to master=" + masterServerName + " with port="
2021         + rpcServices.isa.getPort() + ", startcode=" + this.startcode);
2022       long now = EnvironmentEdgeManager.currentTimeMillis();
2023       int port = rpcServices.isa.getPort();
2024       RegionServerStartupRequest.Builder request = RegionServerStartupRequest.newBuilder();
2025       request.setPort(port);
2026       request.setServerStartCode(this.startcode);
2027       request.setServerCurrentTime(now);
2028       result = this.rssStub.regionServerStartup(null, request.build());
2029     } catch (ServiceException se) {
2030       IOException ioe = ProtobufUtil.getRemoteException(se);
2031       if (ioe instanceof ClockOutOfSyncException) {
2032         LOG.fatal("Master rejected startup because clock is out of sync", ioe);
2033         // Re-throw IOE will cause RS to abort
2034         throw ioe;
2035       } else if (ioe instanceof ServerNotRunningYetException) {
2036         LOG.debug("Master is not running yet");
2037       } else {
2038         LOG.warn("error telling master we are up", se);
2039       }
2040     }
2041     return result;
2042   }
2043 
2044   @Override
2045   public long getLastSequenceId(byte[] region) {
2046     Long lastFlushedSequenceId = -1l;
2047     try {
2048       GetLastFlushedSequenceIdRequest req = RequestConverter
2049           .buildGetLastFlushedSequenceIdRequest(region);
2050       lastFlushedSequenceId = rssStub.getLastFlushedSequenceId(null, req)
2051           .getLastFlushedSequenceId();
2052     } catch (ServiceException e) {
2053       lastFlushedSequenceId = -1l;
2054       LOG.warn("Unable to connect to the master to check " + "the last flushed sequence id", e);
2055     }
2056     return lastFlushedSequenceId;
2057   }
2058 
2059   /**
2060    * Closes all regions.  Called on our way out.
2061    * Assumes that its not possible for new regions to be added to onlineRegions
2062    * while this method runs.
2063    */
2064   protected void closeAllRegions(final boolean abort) {
2065     closeUserRegions(abort);
2066     closeMetaTableRegions(abort);
2067   }
2068 
2069   /**
2070    * Close meta region if we carry it
2071    * @param abort Whether we're running an abort.
2072    */
2073   void closeMetaTableRegions(final boolean abort) {
2074     HRegion meta = null;
2075     this.lock.writeLock().lock();
2076     try {
2077       for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
2078         HRegionInfo hri = e.getValue().getRegionInfo();
2079         if (hri.isMetaRegion()) {
2080           meta = e.getValue();
2081         }
2082         if (meta != null) break;
2083       }
2084     } finally {
2085       this.lock.writeLock().unlock();
2086     }
2087     if (meta != null) closeRegionIgnoreErrors(meta.getRegionInfo(), abort);
2088   }
2089 
2090   /**
2091    * Schedule closes on all user regions.
2092    * Should be safe calling multiple times because it wont' close regions
2093    * that are already closed or that are closing.
2094    * @param abort Whether we're running an abort.
2095    */
2096   void closeUserRegions(final boolean abort) {
2097     this.lock.writeLock().lock();
2098     try {
2099       for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
2100         HRegion r = e.getValue();
2101         if (!r.getRegionInfo().isMetaTable() && r.isAvailable()) {
2102           // Don't update zk with this close transition; pass false.
2103           closeRegionIgnoreErrors(r.getRegionInfo(), abort);
2104         }
2105       }
2106     } finally {
2107       this.lock.writeLock().unlock();
2108     }
2109   }
2110 
2111   /** @return the info server */
2112   public InfoServer getInfoServer() {
2113     return infoServer;
2114   }
2115 
2116   /**
2117    * @return true if a stop has been requested.
2118    */
2119   @Override
2120   public boolean isStopped() {
2121     return this.stopped;
2122   }
2123 
2124   @Override
2125   public boolean isStopping() {
2126     return this.stopping;
2127   }
2128 
2129   @Override
2130   public Map<String, HRegion> getRecoveringRegions() {
2131     return this.recoveringRegions;
2132   }
2133 
2134   /**
2135    *
2136    * @return the configuration
2137    */
2138   @Override
2139   public Configuration getConfiguration() {
2140     return conf;
2141   }
2142 
2143   /** @return the write lock for the server */
2144   ReentrantReadWriteLock.WriteLock getWriteLock() {
2145     return lock.writeLock();
2146   }
2147 
2148   public int getNumberOfOnlineRegions() {
2149     return this.onlineRegions.size();
2150   }
2151 
2152   boolean isOnlineRegionsEmpty() {
2153     return this.onlineRegions.isEmpty();
2154   }
2155 
2156   /**
2157    * For tests, web ui and metrics.
2158    * This method will only work if HRegionServer is in the same JVM as client;
2159    * HRegion cannot be serialized to cross an rpc.
2160    */
2161   public Collection<HRegion> getOnlineRegionsLocalContext() {
2162     Collection<HRegion> regions = this.onlineRegions.values();
2163     return Collections.unmodifiableCollection(regions);
2164   }
2165 
2166   @Override
2167   public void addToOnlineRegions(HRegion region) {
2168     this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
2169   }
2170 
2171   /**
2172    * @return A new Map of online regions sorted by region size with the first entry being the
2173    * biggest.  If two regions are the same size, then the last one found wins; i.e. this method
2174    * may NOT return all regions.
2175    */
2176   SortedMap<Long, HRegion> getCopyOfOnlineRegionsSortedBySize() {
2177     // we'll sort the regions in reverse
2178     SortedMap<Long, HRegion> sortedRegions = new TreeMap<Long, HRegion>(
2179         new Comparator<Long>() {
2180           @Override
2181           public int compare(Long a, Long b) {
2182             return -1 * a.compareTo(b);
2183           }
2184         });
2185     // Copy over all regions. Regions are sorted by size with biggest first.
2186     for (HRegion region : this.onlineRegions.values()) {
2187       sortedRegions.put(region.memstoreSize.get(), region);
2188     }
2189     return sortedRegions;
2190   }
2191 
2192   /**
2193    * @return time stamp in millis of when this region server was started
2194    */
2195   public long getStartcode() {
2196     return this.startcode;
2197   }
2198 
2199   /** @return reference to FlushRequester */
2200   @Override
2201   public FlushRequester getFlushRequester() {
2202     return this.cacheFlusher;
2203   }
2204 
2205   /**
2206    * Get the top N most loaded regions this server is serving so we can tell the
2207    * master which regions it can reallocate if we're overloaded. TODO: actually
2208    * calculate which regions are most loaded. (Right now, we're just grabbing
2209    * the first N regions being served regardless of load.)
2210    */
2211   protected HRegionInfo[] getMostLoadedRegions() {
2212     ArrayList<HRegionInfo> regions = new ArrayList<HRegionInfo>();
2213     for (HRegion r : onlineRegions.values()) {
2214       if (!r.isAvailable()) {
2215         continue;
2216       }
2217       if (regions.size() < numRegionsToReport) {
2218         regions.add(r.getRegionInfo());
2219       } else {
2220         break;
2221       }
2222     }
2223     return regions.toArray(new HRegionInfo[regions.size()]);
2224   }
2225 
2226   @Override
2227   public Leases getLeases() {
2228     return leases;
2229   }
2230 
2231   /**
2232    * @return Return the rootDir.
2233    */
2234   protected Path getRootDir() {
2235     return rootDir;
2236   }
2237 
2238   /**
2239    * @return Return the fs.
2240    */
2241   @Override
2242   public FileSystem getFileSystem() {
2243     return fs;
2244   }
2245 
2246   @Override
2247   public String toString() {
2248     return getServerName().toString();
2249   }
2250 
2251   /**
2252    * Interval at which threads should run
2253    *
2254    * @return the interval
2255    */
2256   public int getThreadWakeFrequency() {
2257     return threadWakeFrequency;
2258   }
2259 
2260   @Override
2261   public ZooKeeperWatcher getZooKeeper() {
2262     return zooKeeper;
2263   }
2264 
2265   @Override
2266   public BaseCoordinatedStateManager getCoordinatedStateManager() {
2267     return csm;
2268   }
2269 
2270   @Override
2271   public ServerName getServerName() {
2272     return serverName;
2273   }
2274 
2275   @Override
2276   public CompactionRequestor getCompactionRequester() {
2277     return this.compactSplitThread;
2278   }
2279 
2280   public RegionServerCoprocessorHost getRegionServerCoprocessorHost(){
2281     return this.rsHost;
2282   }
2283 
2284   @Override
2285   public ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS() {
2286     return this.regionsInTransitionInRS;
2287   }
2288 
2289   @Override
2290   public ExecutorService getExecutorService() {
2291     return service;
2292   }
2293 
2294   //
2295   // Main program and support routines
2296   //
2297 
2298   /**
2299    * Load the replication service objects, if any
2300    */
2301   static private void createNewReplicationInstance(Configuration conf,
2302     HRegionServer server, FileSystem fs, Path logDir, Path oldLogDir) throws IOException{
2303 
2304     // If replication is not enabled, then return immediately.
2305     if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
2306         HConstants.REPLICATION_ENABLE_DEFAULT)) {
2307       return;
2308     }
2309 
2310     // read in the name of the source replication class from the config file.
2311     String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
2312                                HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
2313 
2314     // read in the name of the sink replication class from the config file.
2315     String sinkClassname = conf.get(HConstants.REPLICATION_SINK_SERVICE_CLASSNAME,
2316                              HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
2317 
2318     // If both the sink and the source class names are the same, then instantiate
2319     // only one object.
2320     if (sourceClassname.equals(sinkClassname)) {
2321       server.replicationSourceHandler = (ReplicationSourceService)
2322                                          newReplicationInstance(sourceClassname,
2323                                          conf, server, fs, logDir, oldLogDir);
2324       server.replicationSinkHandler = (ReplicationSinkService)
2325                                          server.replicationSourceHandler;
2326     } else {
2327       server.replicationSourceHandler = (ReplicationSourceService)
2328                                          newReplicationInstance(sourceClassname,
2329                                          conf, server, fs, logDir, oldLogDir);
2330       server.replicationSinkHandler = (ReplicationSinkService)
2331                                          newReplicationInstance(sinkClassname,
2332                                          conf, server, fs, logDir, oldLogDir);
2333     }
2334   }
2335 
2336   static private ReplicationService newReplicationInstance(String classname,
2337     Configuration conf, HRegionServer server, FileSystem fs, Path logDir,
2338     Path oldLogDir) throws IOException{
2339 
2340     Class<?> clazz = null;
2341     try {
2342       ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
2343       clazz = Class.forName(classname, true, classLoader);
2344     } catch (java.lang.ClassNotFoundException nfe) {
2345       throw new IOException("Could not find class for " + classname);
2346     }
2347 
2348     // create an instance of the replication object.
2349     ReplicationService service = (ReplicationService)
2350                               ReflectionUtils.newInstance(clazz, conf);
2351     service.initialize(server, fs, logDir, oldLogDir);
2352     return service;
2353   }
2354 
2355   /**
2356    * Utility for constructing an instance of the passed HRegionServer class.
2357    *
2358    * @param regionServerClass
2359    * @param conf2
2360    * @return HRegionServer instance.
2361    */
2362   public static HRegionServer constructRegionServer(
2363       Class<? extends HRegionServer> regionServerClass,
2364       final Configuration conf2, CoordinatedStateManager cp) {
2365     try {
2366       Constructor<? extends HRegionServer> c = regionServerClass
2367           .getConstructor(Configuration.class, CoordinatedStateManager.class);
2368       return c.newInstance(conf2, cp);
2369     } catch (Exception e) {
2370       throw new RuntimeException("Failed construction of " + "Regionserver: "
2371           + regionServerClass.toString(), e);
2372     }
2373   }
2374 
2375   /**
2376    * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
2377    */
2378   public static void main(String[] args) throws Exception {
2379 	VersionInfo.logVersion();
2380     Configuration conf = HBaseConfiguration.create();
2381     @SuppressWarnings("unchecked")
2382     Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
2383         .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
2384 
2385     new HRegionServerCommandLine(regionServerClass).doMain(args);
2386   }
2387 
2388   /**
2389    * Gets the online regions of the specified table.
2390    * This method looks at the in-memory onlineRegions.  It does not go to <code>hbase:meta</code>.
2391    * Only returns <em>online</em> regions.  If a region on this table has been
2392    * closed during a disable, etc., it will not be included in the returned list.
2393    * So, the returned list may not necessarily be ALL regions in this table, its
2394    * all the ONLINE regions in the table.
2395    * @param tableName
2396    * @return Online regions from <code>tableName</code>
2397    */
2398   @Override
2399   public List<HRegion> getOnlineRegions(TableName tableName) {
2400      List<HRegion> tableRegions = new ArrayList<HRegion>();
2401      synchronized (this.onlineRegions) {
2402        for (HRegion region: this.onlineRegions.values()) {
2403          HRegionInfo regionInfo = region.getRegionInfo();
2404          if(regionInfo.getTable().equals(tableName)) {
2405            tableRegions.add(region);
2406          }
2407        }
2408      }
2409      return tableRegions;
2410    }
2411 
2412   // used by org/apache/hbase/tmpl/regionserver/RSStatusTmpl.jamon (HBASE-4070).
2413   public String[] getRegionServerCoprocessors() {
2414     TreeSet<String> coprocessors = new TreeSet<String>(
2415         this.hlog.getCoprocessorHost().getCoprocessors());
2416     Collection<HRegion> regions = getOnlineRegionsLocalContext();
2417     for (HRegion region: regions) {
2418       coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
2419     }
2420     return coprocessors.toArray(new String[coprocessors.size()]);
2421   }
2422 
2423   /**
2424    * Try to close the region, logs a warning on failure but continues.
2425    * @param region Region to close
2426    */
2427   private void closeRegionIgnoreErrors(HRegionInfo region, final boolean abort) {
2428     try {
2429       CloseRegionCoordination.CloseRegionDetails details =
2430         csm.getCloseRegionCoordination().getDetaultDetails();
2431       if (!closeRegion(region.getEncodedName(), abort, details, null)) {
2432         LOG.warn("Failed to close " + region.getRegionNameAsString() +
2433             " - ignoring and continuing");
2434       }
2435     } catch (IOException e) {
2436       LOG.warn("Failed to close " + region.getRegionNameAsString() +
2437           " - ignoring and continuing", e);
2438     }
2439   }
2440 
2441   /**
2442    * Close asynchronously a region, can be called from the master or internally by the regionserver
2443    * when stopping. If called from the master, the region will update the znode status.
2444    *
2445    * <p>
2446    * If an opening was in progress, this method will cancel it, but will not start a new close. The
2447    * coprocessors are not called in this case. A NotServingRegionException exception is thrown.
2448    * </p>
2449 
2450    * <p>
2451    *   If a close was in progress, this new request will be ignored, and an exception thrown.
2452    * </p>
2453    *
2454    * @param encodedName Region to close
2455    * @param abort True if we are aborting
2456    * @param crd details about closing region coordination-coordinated task
2457    * @return True if closed a region.
2458    * @throws NotServingRegionException if the region is not online
2459    * @throws RegionAlreadyInTransitionException if the region is already closing
2460    */
2461   protected boolean closeRegion(String encodedName, final boolean abort,
2462       CloseRegionCoordination.CloseRegionDetails crd, final ServerName sn)
2463       throws NotServingRegionException, RegionAlreadyInTransitionException {
2464     //Check for permissions to close.
2465     HRegion actualRegion = this.getFromOnlineRegions(encodedName);
2466     if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
2467       try {
2468         actualRegion.getCoprocessorHost().preClose(false);
2469       } catch (IOException exp) {
2470         LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
2471         return false;
2472       }
2473     }
2474 
2475     final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(encodedName.getBytes(),
2476         Boolean.FALSE);
2477 
2478     if (Boolean.TRUE.equals(previous)) {
2479       LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
2480           "trying to OPEN. Cancelling OPENING.");
2481       if (!regionsInTransitionInRS.replace(encodedName.getBytes(), previous, Boolean.FALSE)){
2482         // The replace failed. That should be an exceptional case, but theoretically it can happen.
2483         // We're going to try to do a standard close then.
2484         LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
2485             " Doing a standard close now");
2486         return closeRegion(encodedName, abort, crd, sn);
2487       }
2488       // Let's get the region from the online region list again
2489       actualRegion = this.getFromOnlineRegions(encodedName);
2490       if (actualRegion == null) { // If already online, we still need to close it.
2491         LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
2492         // The master deletes the znode when it receives this exception.
2493         throw new NotServingRegionException("The region " + encodedName +
2494           " was opening but not yet served. Opening is cancelled.");
2495       }
2496     } else if (Boolean.FALSE.equals(previous)) {
2497       LOG.info("Received CLOSE for the region: " + encodedName +
2498         " ,which we are already trying to CLOSE, but not completed yet");
2499       // The master will retry till the region is closed. We need to do this since
2500       // the region could fail to close somehow. If we mark the region closed in master
2501       // while it is not, there could be data loss.
2502       // If the region stuck in closing for a while, and master runs out of retries,
2503       // master will move the region to failed_to_close. Later on, if the region
2504       // is indeed closed, master can properly re-assign it.
2505       throw new RegionAlreadyInTransitionException("The region " + encodedName +
2506         " was already closing. New CLOSE request is ignored.");
2507     }
2508 
2509     if (actualRegion == null) {
2510       LOG.error("Received CLOSE for a region which is not online, and we're not opening.");
2511       this.regionsInTransitionInRS.remove(encodedName.getBytes());
2512       // The master deletes the znode when it receives this exception.
2513       throw new NotServingRegionException("The region " + encodedName +
2514           " is not online, and is not opening.");
2515     }
2516 
2517     CloseRegionHandler crh;
2518     final HRegionInfo hri = actualRegion.getRegionInfo();
2519     if (hri.isMetaRegion()) {
2520       crh = new CloseMetaHandler(this, this, hri, abort,
2521         csm.getCloseRegionCoordination(), crd);
2522     } else {
2523       crh = new CloseRegionHandler(this, this, hri, abort,
2524         csm.getCloseRegionCoordination(), crd, sn);
2525     }
2526     this.service.submit(crh);
2527     return true;
2528   }
2529 
2530    /**
2531    * @param regionName
2532    * @return HRegion for the passed binary <code>regionName</code> or null if
2533    *         named region is not member of the online regions.
2534    */
2535   public HRegion getOnlineRegion(final byte[] regionName) {
2536     String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
2537     return this.onlineRegions.get(encodedRegionName);
2538   }
2539 
2540   public InetSocketAddress[] getRegionBlockLocations(final String encodedRegionName) {
2541     return this.regionFavoredNodesMap.get(encodedRegionName);
2542   }
2543 
2544   @Override
2545   public HRegion getFromOnlineRegions(final String encodedRegionName) {
2546     return this.onlineRegions.get(encodedRegionName);
2547   }
2548 
2549 
2550   @Override
2551   public boolean removeFromOnlineRegions(final HRegion r, ServerName destination) {
2552     HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
2553 
2554     if (destination != null) {
2555       HLog wal = getWAL();
2556       long closeSeqNum = wal.getEarliestMemstoreSeqNum(r.getRegionInfo().getEncodedNameAsBytes());
2557       if (closeSeqNum == HConstants.NO_SEQNUM) {
2558         // No edits in WAL for this region; get the sequence number when the region was opened.
2559         closeSeqNum = r.getOpenSeqNum();
2560         if (closeSeqNum == HConstants.NO_SEQNUM) {
2561           closeSeqNum = 0;
2562         }
2563       }
2564       addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum);
2565     }
2566     this.regionFavoredNodesMap.remove(r.getRegionInfo().getEncodedName());
2567     return toReturn != null;
2568   }
2569 
2570   /**
2571    * Protected utility method for safely obtaining an HRegion handle.
2572    *
2573    * @param regionName
2574    *          Name of online {@link HRegion} to return
2575    * @return {@link HRegion} for <code>regionName</code>
2576    * @throws NotServingRegionException
2577    */
2578   protected HRegion getRegion(final byte[] regionName)
2579       throws NotServingRegionException {
2580     String encodedRegionName = HRegionInfo.encodeRegionName(regionName);
2581     return getRegionByEncodedName(regionName, encodedRegionName);
2582   }
2583 
2584   public HRegion getRegionByEncodedName(String encodedRegionName)
2585       throws NotServingRegionException {
2586     return getRegionByEncodedName(null, encodedRegionName);
2587   }
2588 
2589   protected HRegion getRegionByEncodedName(byte[] regionName, String encodedRegionName)
2590     throws NotServingRegionException {
2591     HRegion region = this.onlineRegions.get(encodedRegionName);
2592     if (region == null) {
2593       MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
2594       if (moveInfo != null) {
2595         throw new RegionMovedException(moveInfo.getServerName(), moveInfo.getSeqNum());
2596       }
2597       Boolean isOpening = this.regionsInTransitionInRS.get(Bytes.toBytes(encodedRegionName));
2598       String regionNameStr = regionName == null?
2599         encodedRegionName: Bytes.toStringBinary(regionName);
2600       if (isOpening != null && isOpening.booleanValue()) {
2601         throw new RegionOpeningException("Region " + regionNameStr +
2602           " is opening on " + this.serverName);
2603       }
2604       throw new NotServingRegionException("Region " + regionNameStr +
2605         " is not online on " + this.serverName);
2606     }
2607     return region;
2608   }
2609 
2610   /*
2611    * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
2612    * IOE if it isn't already.
2613    *
2614    * @param t Throwable
2615    *
2616    * @param msg Message to log in error. Can be null.
2617    *
2618    * @return Throwable converted to an IOE; methods can only let out IOEs.
2619    */
2620   private Throwable cleanup(final Throwable t, final String msg) {
2621     // Don't log as error if NSRE; NSRE is 'normal' operation.
2622     if (t instanceof NotServingRegionException) {
2623       LOG.debug("NotServingRegionException; " + t.getMessage());
2624       return t;
2625     }
2626     if (msg == null) {
2627       LOG.error("", RemoteExceptionHandler.checkThrowable(t));
2628     } else {
2629       LOG.error(msg, RemoteExceptionHandler.checkThrowable(t));
2630     }
2631     if (!rpcServices.checkOOME(t)) {
2632       checkFileSystem();
2633     }
2634     return t;
2635   }
2636 
2637   /*
2638    * @param t
2639    *
2640    * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
2641    *
2642    * @return Make <code>t</code> an IOE if it isn't already.
2643    */
2644   protected IOException convertThrowableToIOE(final Throwable t, final String msg) {
2645     return (t instanceof IOException ? (IOException) t : msg == null
2646         || msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
2647   }
2648 
2649   /**
2650    * Checks to see if the file system is still accessible. If not, sets
2651    * abortRequested and stopRequested
2652    *
2653    * @return false if file system is not available
2654    */
2655   public boolean checkFileSystem() {
2656     if (this.fsOk && this.fs != null) {
2657       try {
2658         FSUtils.checkFileSystemAvailable(this.fs);
2659       } catch (IOException e) {
2660         abort("File System not available", e);
2661         this.fsOk = false;
2662       }
2663     }
2664     return this.fsOk;
2665   }
2666 
2667   @Override
2668   public void updateRegionFavoredNodesMapping(String encodedRegionName,
2669       List<org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName> favoredNodes) {
2670     InetSocketAddress[] addr = new InetSocketAddress[favoredNodes.size()];
2671     // Refer to the comment on the declaration of regionFavoredNodesMap on why
2672     // it is a map of region name to InetSocketAddress[]
2673     for (int i = 0; i < favoredNodes.size(); i++) {
2674       addr[i] = InetSocketAddress.createUnresolved(favoredNodes.get(i).getHostName(),
2675           favoredNodes.get(i).getPort());
2676     }
2677     regionFavoredNodesMap.put(encodedRegionName, addr);
2678   }
2679 
2680   /**
2681    * Return the favored nodes for a region given its encoded name. Look at the
2682    * comment around {@link #regionFavoredNodesMap} on why it is InetSocketAddress[]
2683    * @param encodedRegionName
2684    * @return array of favored locations
2685    */
2686   @Override
2687   public InetSocketAddress[] getFavoredNodesForRegion(String encodedRegionName) {
2688     return regionFavoredNodesMap.get(encodedRegionName);
2689   }
2690 
2691   @Override
2692   public ServerNonceManager getNonceManager() {
2693     return this.nonceManager;
2694   }
2695 
2696   private static class MovedRegionInfo {
2697     private final ServerName serverName;
2698     private final long seqNum;
2699     private final long ts;
2700 
2701     public MovedRegionInfo(ServerName serverName, long closeSeqNum) {
2702       this.serverName = serverName;
2703       this.seqNum = closeSeqNum;
2704       ts = EnvironmentEdgeManager.currentTimeMillis();
2705      }
2706 
2707     public ServerName getServerName() {
2708       return serverName;
2709     }
2710 
2711     public long getSeqNum() {
2712       return seqNum;
2713     }
2714 
2715     public long getMoveTime() {
2716       return ts;
2717     }
2718   }
2719 
2720   // This map will contains all the regions that we closed for a move.
2721   //  We add the time it was moved as we don't want to keep too old information
2722   protected Map<String, MovedRegionInfo> movedRegions =
2723       new ConcurrentHashMap<String, MovedRegionInfo>(3000);
2724 
2725   // We need a timeout. If not there is a risk of giving a wrong information: this would double
2726   //  the number of network calls instead of reducing them.
2727   private static final int TIMEOUT_REGION_MOVED = (2 * 60 * 1000);
2728 
2729   protected void addToMovedRegions(String encodedName, ServerName destination, long closeSeqNum) {
2730     if (ServerName.isSameHostnameAndPort(destination, this.getServerName())) {
2731       LOG.warn("Not adding moved region record: " + encodedName + " to self.");
2732       return;
2733     }
2734     LOG.info("Adding moved region record: "
2735       + encodedName + " to " + destination + " as of " + closeSeqNum);
2736     movedRegions.put(encodedName, new MovedRegionInfo(destination, closeSeqNum));
2737   }
2738 
2739   void removeFromMovedRegions(String encodedName) {
2740     movedRegions.remove(encodedName);
2741   }
2742 
2743   private MovedRegionInfo getMovedRegion(final String encodedRegionName) {
2744     MovedRegionInfo dest = movedRegions.get(encodedRegionName);
2745 
2746     long now = EnvironmentEdgeManager.currentTimeMillis();
2747     if (dest != null) {
2748       if (dest.getMoveTime() > (now - TIMEOUT_REGION_MOVED)) {
2749         return dest;
2750       } else {
2751         movedRegions.remove(encodedRegionName);
2752       }
2753     }
2754 
2755     return null;
2756   }
2757 
2758   /**
2759    * Remove the expired entries from the moved regions list.
2760    */
2761   protected void cleanMovedRegions() {
2762     final long cutOff = System.currentTimeMillis() - TIMEOUT_REGION_MOVED;
2763     Iterator<Entry<String, MovedRegionInfo>> it = movedRegions.entrySet().iterator();
2764 
2765     while (it.hasNext()){
2766       Map.Entry<String, MovedRegionInfo> e = it.next();
2767       if (e.getValue().getMoveTime() < cutOff) {
2768         it.remove();
2769       }
2770     }
2771   }
2772 
2773   /**
2774    * Creates a Chore thread to clean the moved region cache.
2775    */
2776   protected static class MovedRegionsCleaner extends Chore implements Stoppable {
2777     private HRegionServer regionServer;
2778     Stoppable stoppable;
2779 
2780     private MovedRegionsCleaner(
2781       HRegionServer regionServer, Stoppable stoppable){
2782       super("MovedRegionsCleaner for region "+regionServer, TIMEOUT_REGION_MOVED, stoppable);
2783       this.regionServer = regionServer;
2784       this.stoppable = stoppable;
2785     }
2786 
2787     static MovedRegionsCleaner createAndStart(HRegionServer rs){
2788       Stoppable stoppable = new Stoppable() {
2789         private volatile boolean isStopped = false;
2790         @Override public void stop(String why) { isStopped = true;}
2791         @Override public boolean isStopped() {return isStopped;}
2792       };
2793 
2794       return new MovedRegionsCleaner(rs, stoppable);
2795     }
2796 
2797     @Override
2798     protected void chore() {
2799       regionServer.cleanMovedRegions();
2800     }
2801 
2802     @Override
2803     public void stop(String why) {
2804       stoppable.stop(why);
2805     }
2806 
2807     @Override
2808     public boolean isStopped() {
2809       return stoppable.isStopped();
2810     }
2811   }
2812 
2813   private String getMyEphemeralNodePath() {
2814     return ZKUtil.joinZNode(this.zooKeeper.rsZNode, getServerName().toString());
2815   }
2816 
2817   private boolean isHealthCheckerConfigured() {
2818     String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
2819     return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
2820   }
2821 
2822   /**
2823    * @return the underlying {@link CompactSplitThread} for the servers
2824    */
2825   public CompactSplitThread getCompactSplitThread() {
2826     return this.compactSplitThread;
2827   }
2828 
2829   /**
2830    * A helper function to store the last flushed sequence Id with the previous failed RS for a
2831    * recovering region. The Id is used to skip wal edits which are flushed. Since the flushed
2832    * sequence id is only valid for each RS, we associate the Id with corresponding failed RS.
2833    * @throws KeeperException
2834    * @throws IOException
2835    */
2836   private void updateRecoveringRegionLastFlushedSequenceId(HRegion r) throws KeeperException,
2837       IOException {
2838     if (!r.isRecovering()) {
2839       // return immdiately for non-recovering regions
2840       return;
2841     }
2842 
2843     HRegionInfo region = r.getRegionInfo();
2844     ZooKeeperWatcher zkw = getZooKeeper();
2845     String previousRSName = this.getLastFailedRSFromZK(region.getEncodedName());
2846     Map<byte[], Long> maxSeqIdInStores = r.getMaxStoreSeqIdForLogReplay();
2847     long minSeqIdForLogReplay = -1;
2848     for (Long storeSeqIdForReplay : maxSeqIdInStores.values()) {
2849       if (minSeqIdForLogReplay == -1 || storeSeqIdForReplay < minSeqIdForLogReplay) {
2850         minSeqIdForLogReplay = storeSeqIdForReplay;
2851       }
2852     }
2853     long lastRecordedFlushedSequenceId = -1;
2854     String nodePath = ZKUtil.joinZNode(this.zooKeeper.recoveringRegionsZNode,
2855       region.getEncodedName());
2856     // recovering-region level
2857     byte[] data;
2858     try {
2859       data = ZKUtil.getData(zkw, nodePath);
2860     } catch (InterruptedException e) {
2861       throw new InterruptedIOException();
2862     }
2863     if (data != null) {
2864       lastRecordedFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data);
2865     }
2866     if (data == null || lastRecordedFlushedSequenceId < minSeqIdForLogReplay) {
2867       ZKUtil.setData(zkw, nodePath, ZKUtil.positionToByteArray(minSeqIdForLogReplay));
2868     }
2869     if (previousRSName != null) {
2870       // one level deeper for the failed RS
2871       nodePath = ZKUtil.joinZNode(nodePath, previousRSName);
2872       ZKUtil.setData(zkw, nodePath,
2873         ZKUtil.regionSequenceIdsToByteArray(minSeqIdForLogReplay, maxSeqIdInStores));
2874       LOG.debug("Update last flushed sequence id of region " + region.getEncodedName() + " for "
2875           + previousRSName);
2876     } else {
2877       LOG.warn("Can't find failed region server for recovering region " + region.getEncodedName());
2878     }
2879   }
2880 
2881   /**
2882    * Return the last failed RS name under /hbase/recovering-regions/encodedRegionName
2883    * @param encodedRegionName
2884    * @throws KeeperException
2885    */
2886   private String getLastFailedRSFromZK(String encodedRegionName) throws KeeperException {
2887     String result = null;
2888     long maxZxid = 0;
2889     ZooKeeperWatcher zkw = this.getZooKeeper();
2890     String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName);
2891     List<String> failedServers = ZKUtil.listChildrenNoWatch(zkw, nodePath);
2892     if (failedServers == null || failedServers.isEmpty()) {
2893       return result;
2894     }
2895     for (String failedServer : failedServers) {
2896       String rsPath = ZKUtil.joinZNode(nodePath, failedServer);
2897       Stat stat = new Stat();
2898       ZKUtil.getDataNoWatch(zkw, rsPath, stat);
2899       if (maxZxid < stat.getCzxid()) {
2900         maxZxid = stat.getCzxid();
2901         result = failedServer;
2902       }
2903     }
2904     return result;
2905   }
2906 
2907   /**
2908    * @return The cache config instance used by the regionserver.
2909    */
2910   public CacheConfig getCacheConfig() {
2911     return this.cacheConfig;
2912   }
2913 }