View Javadoc

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