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