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