View Javadoc

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