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