View Javadoc

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