View Javadoc

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