001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.regionserver;
019
020import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK;
021import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_WAL_MAX_SPLITTER;
022import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK;
023import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_MAX_SPLITTER;
024import static org.apache.hadoop.hbase.util.DNS.RS_HOSTNAME_KEY;
025
026import java.io.IOException;
027import java.lang.management.MemoryType;
028import java.lang.management.MemoryUsage;
029import java.lang.reflect.Constructor;
030import java.net.BindException;
031import java.net.InetAddress;
032import java.net.InetSocketAddress;
033import java.time.Duration;
034import java.util.ArrayList;
035import java.util.Collection;
036import java.util.Collections;
037import java.util.Comparator;
038import java.util.HashSet;
039import java.util.List;
040import java.util.Map;
041import java.util.Map.Entry;
042import java.util.Objects;
043import java.util.Optional;
044import java.util.Set;
045import java.util.SortedMap;
046import java.util.Timer;
047import java.util.TimerTask;
048import java.util.TreeMap;
049import java.util.TreeSet;
050import java.util.concurrent.ConcurrentHashMap;
051import java.util.concurrent.ConcurrentMap;
052import java.util.concurrent.ConcurrentSkipListMap;
053import java.util.concurrent.TimeUnit;
054import java.util.concurrent.atomic.AtomicBoolean;
055import java.util.concurrent.locks.ReentrantReadWriteLock;
056import java.util.stream.Collectors;
057import javax.management.MalformedObjectNameException;
058import javax.servlet.http.HttpServlet;
059import org.apache.commons.lang3.RandomUtils;
060import org.apache.commons.lang3.StringUtils;
061import org.apache.commons.lang3.SystemUtils;
062import org.apache.hadoop.conf.Configuration;
063import org.apache.hadoop.fs.FileSystem;
064import org.apache.hadoop.fs.Path;
065import org.apache.hadoop.hbase.Abortable;
066import org.apache.hadoop.hbase.CacheEvictionStats;
067import org.apache.hadoop.hbase.CallQueueTooBigException;
068import org.apache.hadoop.hbase.ChoreService;
069import org.apache.hadoop.hbase.ClockOutOfSyncException;
070import org.apache.hadoop.hbase.CoordinatedStateManager;
071import org.apache.hadoop.hbase.DoNotRetryIOException;
072import org.apache.hadoop.hbase.ExecutorStatusChore;
073import org.apache.hadoop.hbase.HBaseConfiguration;
074import org.apache.hadoop.hbase.HBaseInterfaceAudience;
075import org.apache.hadoop.hbase.HConstants;
076import org.apache.hadoop.hbase.HDFSBlocksDistribution;
077import org.apache.hadoop.hbase.HealthCheckChore;
078import org.apache.hadoop.hbase.MetaTableAccessor;
079import org.apache.hadoop.hbase.NotServingRegionException;
080import org.apache.hadoop.hbase.PleaseHoldException;
081import org.apache.hadoop.hbase.ScheduledChore;
082import org.apache.hadoop.hbase.ServerName;
083import org.apache.hadoop.hbase.Stoppable;
084import org.apache.hadoop.hbase.TableDescriptors;
085import org.apache.hadoop.hbase.TableName;
086import org.apache.hadoop.hbase.YouAreDeadException;
087import org.apache.hadoop.hbase.ZNodeClearer;
088import org.apache.hadoop.hbase.client.AsyncClusterConnection;
089import org.apache.hadoop.hbase.client.ClusterConnectionFactory;
090import org.apache.hadoop.hbase.client.Connection;
091import org.apache.hadoop.hbase.client.ConnectionFactory;
092import org.apache.hadoop.hbase.client.ConnectionUtils;
093import org.apache.hadoop.hbase.client.RegionInfo;
094import org.apache.hadoop.hbase.client.RegionInfoBuilder;
095import org.apache.hadoop.hbase.client.locking.EntityLock;
096import org.apache.hadoop.hbase.client.locking.LockServiceClient;
097import org.apache.hadoop.hbase.conf.ConfigurationManager;
098import org.apache.hadoop.hbase.conf.ConfigurationObserver;
099import org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager;
100import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
101import org.apache.hadoop.hbase.exceptions.RegionMovedException;
102import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
103import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
104import org.apache.hadoop.hbase.executor.ExecutorService;
105import org.apache.hadoop.hbase.executor.ExecutorType;
106import org.apache.hadoop.hbase.fs.HFileSystem;
107import org.apache.hadoop.hbase.http.InfoServer;
108import org.apache.hadoop.hbase.io.hfile.BlockCache;
109import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
110import org.apache.hadoop.hbase.io.hfile.HFile;
111import org.apache.hadoop.hbase.io.util.MemorySizeUtil;
112import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
113import org.apache.hadoop.hbase.ipc.NettyRpcClientConfigHelper;
114import org.apache.hadoop.hbase.ipc.RpcClient;
115import org.apache.hadoop.hbase.ipc.RpcServer;
116import org.apache.hadoop.hbase.ipc.RpcServerInterface;
117import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
118import org.apache.hadoop.hbase.ipc.ServerRpcController;
119import org.apache.hadoop.hbase.log.HBaseMarkers;
120import org.apache.hadoop.hbase.master.HMaster;
121import org.apache.hadoop.hbase.master.LoadBalancer;
122import org.apache.hadoop.hbase.master.MasterRpcServicesVersionWrapper;
123import org.apache.hadoop.hbase.master.RegionState;
124import org.apache.hadoop.hbase.mob.MobFileCache;
125import org.apache.hadoop.hbase.procedure.RegionServerProcedureManagerHost;
126import org.apache.hadoop.hbase.procedure2.RSProcedureCallable;
127import org.apache.hadoop.hbase.quotas.FileSystemUtilizationChore;
128import org.apache.hadoop.hbase.quotas.QuotaUtil;
129import org.apache.hadoop.hbase.quotas.RegionServerRpcQuotaManager;
130import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
131import org.apache.hadoop.hbase.quotas.RegionSize;
132import org.apache.hadoop.hbase.quotas.RegionSizeStore;
133import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
134import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
135import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
136import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequester;
137import org.apache.hadoop.hbase.regionserver.handler.CloseMetaHandler;
138import org.apache.hadoop.hbase.regionserver.handler.CloseRegionHandler;
139import org.apache.hadoop.hbase.regionserver.handler.RSProcedureHandler;
140import org.apache.hadoop.hbase.regionserver.handler.RegionReplicaFlushHandler;
141import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
142import org.apache.hadoop.hbase.namequeues.SlowLogTableOpsChore;
143import org.apache.hadoop.hbase.regionserver.throttle.FlushThroughputControllerFactory;
144import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
145import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
146import org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceInterface;
147import org.apache.hadoop.hbase.replication.regionserver.ReplicationStatus;
148import org.apache.hadoop.hbase.security.SecurityConstants;
149import org.apache.hadoop.hbase.security.Superusers;
150import org.apache.hadoop.hbase.security.User;
151import org.apache.hadoop.hbase.security.UserProvider;
152import org.apache.hadoop.hbase.security.access.AccessChecker;
153import org.apache.hadoop.hbase.security.access.ZKPermissionWatcher;
154import org.apache.hadoop.hbase.trace.SpanReceiverHost;
155import org.apache.hadoop.hbase.trace.TraceUtil;
156import org.apache.hadoop.hbase.util.Addressing;
157import org.apache.hadoop.hbase.util.Bytes;
158import org.apache.hadoop.hbase.util.CommonFSUtils;
159import org.apache.hadoop.hbase.util.CompressionTest;
160import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
161import org.apache.hadoop.hbase.util.FSTableDescriptors;
162import org.apache.hadoop.hbase.util.FSUtils;
163import org.apache.hadoop.hbase.util.FutureUtils;
164import org.apache.hadoop.hbase.util.JvmPauseMonitor;
165import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
166import org.apache.hadoop.hbase.util.Pair;
167import org.apache.hadoop.hbase.util.RetryCounter;
168import org.apache.hadoop.hbase.util.RetryCounterFactory;
169import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
170import org.apache.hadoop.hbase.util.Sleeper;
171import org.apache.hadoop.hbase.util.Threads;
172import org.apache.hadoop.hbase.util.VersionInfo;
173import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
174import org.apache.hadoop.hbase.wal.NettyAsyncFSWALConfigHelper;
175import org.apache.hadoop.hbase.wal.WAL;
176import org.apache.hadoop.hbase.wal.WALFactory;
177import org.apache.hadoop.hbase.wal.WALProvider;
178import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
179import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
180import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
181import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
182import org.apache.hadoop.hbase.zookeeper.ZKNodeTracker;
183import org.apache.hadoop.hbase.zookeeper.ZKUtil;
184import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
185import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
186import org.apache.hadoop.ipc.RemoteException;
187import org.apache.hadoop.util.ReflectionUtils;
188import org.apache.yetus.audience.InterfaceAudience;
189import org.apache.zookeeper.KeeperException;
190import org.slf4j.Logger;
191import org.slf4j.LoggerFactory;
192import sun.misc.Signal;
193
194import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
195import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
196import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
197import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
198import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
199import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
200import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
201import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
202import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
203import org.apache.hbase.thirdparty.com.google.protobuf.Message;
204import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
205import org.apache.hbase.thirdparty.com.google.protobuf.Service;
206import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
207import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
208import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
209
210import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
211import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
212import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
213import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
214import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
215import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
216import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
217import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
218import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.UserLoad;
219import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor;
220import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.Coprocessor.Builder;
221import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.NameStringPair;
222import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionServerInfo;
223import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
224import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
225import org.apache.hadoop.hbase.shaded.protobuf.generated.LockServiceProtos.LockService;
226import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos;
227import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
228import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
229import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
230import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
231import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
232import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionServerStatusService;
233import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUse;
234import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionSpaceUseReportRequest;
235import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
236import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
237import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportProcedureDoneRequest;
238import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
239import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
240import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
241
242/**
243 * HRegionServer makes a set of HRegions available to clients. It checks in with
244 * the HMaster. There are many HRegionServers in a single HBase deployment.
245 */
246@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
247@SuppressWarnings({ "deprecation"})
248public class HRegionServer extends Thread implements
249    RegionServerServices, LastSequenceId, ConfigurationObserver {
250  private static final Logger LOG = LoggerFactory.getLogger(HRegionServer.class);
251
252  /**
253   * For testing only!  Set to true to skip notifying region assignment to master .
254   */
255  @VisibleForTesting
256  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
257  public static boolean TEST_SKIP_REPORTING_TRANSITION = false;
258
259  /**
260   * A map from RegionName to current action in progress. Boolean value indicates:
261   * true - if open region action in progress
262   * false - if close region action in progress
263   */
264  private final ConcurrentMap<byte[], Boolean> regionsInTransitionInRS =
265    new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
266
267  /**
268   * Used to cache the open/close region procedures which already submitted.
269   * See {@link #submitRegionProcedure(long)}.
270   */
271  private final ConcurrentMap<Long, Long> submittedRegionProcedures = new ConcurrentHashMap<>();
272  /**
273   * Used to cache the open/close region procedures which already executed.
274   * See {@link #submitRegionProcedure(long)}.
275   */
276  private final Cache<Long, Long> executedRegionProcedures =
277      CacheBuilder.newBuilder().expireAfterAccess(600, TimeUnit.SECONDS).build();
278
279  private MemStoreFlusher cacheFlusher;
280
281  private HeapMemoryManager hMemManager;
282
283  /**
284   * The asynchronous cluster connection to be shared by services.
285   */
286  protected AsyncClusterConnection asyncClusterConnection;
287
288  /**
289   * Go here to get table descriptors.
290   */
291  protected TableDescriptors tableDescriptors;
292
293  // Replication services. If no replication, this handler will be null.
294  private ReplicationSourceService replicationSourceHandler;
295  private ReplicationSinkService replicationSinkHandler;
296
297  // Compactions
298  public CompactSplit compactSplitThread;
299
300  /**
301   * Map of regions currently being served by this region server. Key is the
302   * encoded region name.  All access should be synchronized.
303   */
304  private final Map<String, HRegion> onlineRegions = new ConcurrentHashMap<>();
305  /**
306   * Lock for gating access to {@link #onlineRegions}.
307   * TODO: If this map is gated by a lock, does it need to be a ConcurrentHashMap?
308   */
309  private final ReentrantReadWriteLock onlineRegionsLock = new ReentrantReadWriteLock();
310
311  /**
312   * Map of encoded region names to the DataNode locations they should be hosted on
313   * We store the value as InetSocketAddress since this is used only in HDFS
314   * API (create() that takes favored nodes as hints for placing file blocks).
315   * We could have used ServerName here as the value class, but we'd need to
316   * convert it to InetSocketAddress at some point before the HDFS API call, and
317   * it seems a bit weird to store ServerName since ServerName refers to RegionServers
318   * and here we really mean DataNode locations.
319   */
320  private final Map<String, InetSocketAddress[]> regionFavoredNodesMap = new ConcurrentHashMap<>();
321
322  private LeaseManager leaseManager;
323
324  // Instance of the hbase executor executorService.
325  protected ExecutorService executorService;
326
327  private volatile boolean dataFsOk;
328  private HFileSystem dataFs;
329  private HFileSystem walFs;
330
331  // Set when a report to the master comes back with a message asking us to
332  // shutdown. Also set by call to stop when debugging or running unit tests
333  // of HRegionServer in isolation.
334  private volatile boolean stopped = false;
335
336  // Go down hard. Used if file system becomes unavailable and also in
337  // debugging and unit tests.
338  private AtomicBoolean abortRequested;
339  static final String ABORT_TIMEOUT = "hbase.regionserver.abort.timeout";
340  // Default abort timeout is 1200 seconds for safe
341  private static final long DEFAULT_ABORT_TIMEOUT = 1200000;
342  // Will run this task when abort timeout
343  static final String ABORT_TIMEOUT_TASK = "hbase.regionserver.abort.timeout.task";
344
345  // A state before we go into stopped state.  At this stage we're closing user
346  // space regions.
347  private boolean stopping = false;
348  private volatile boolean killed = false;
349  private volatile boolean shutDown = false;
350
351  protected final Configuration conf;
352
353  private Path dataRootDir;
354  private Path walRootDir;
355
356  private final int threadWakeFrequency;
357  final int msgInterval;
358
359  private static final String PERIOD_COMPACTION = "hbase.regionserver.compaction.check.period";
360  private final int compactionCheckFrequency;
361  private static final String PERIOD_FLUSH = "hbase.regionserver.flush.check.period";
362  private final int flushCheckFrequency;
363
364  // Stub to do region server status calls against the master.
365  private volatile RegionServerStatusService.BlockingInterface rssStub;
366  private volatile LockService.BlockingInterface lockStub;
367  // RPC client. Used to make the stub above that does region server status checking.
368  private RpcClient rpcClient;
369
370  private UncaughtExceptionHandler uncaughtExceptionHandler;
371
372  // Info server. Default access so can be used by unit tests. REGIONSERVER
373  // is name of the webapp and the attribute name used stuffing this instance
374  // into web context.
375  protected InfoServer infoServer;
376  private JvmPauseMonitor pauseMonitor;
377
378  /** region server process name */
379  public static final String REGIONSERVER = "regionserver";
380
381
382  private MetricsRegionServer metricsRegionServer;
383  MetricsRegionServerWrapperImpl metricsRegionServerImpl;
384  private SpanReceiverHost spanReceiverHost;
385
386  /**
387   * ChoreService used to schedule tasks that we want to run periodically
388   */
389  private ChoreService choreService;
390
391  /**
392   * Check for compactions requests.
393   */
394  private ScheduledChore compactionChecker;
395
396  /**
397   * Check for flushes
398   */
399  private ScheduledChore periodicFlusher;
400
401  private volatile WALFactory walFactory;
402
403  private LogRoller walRoller;
404
405  // A thread which calls reportProcedureDone
406  private RemoteProcedureResultReporter procedureResultReporter;
407
408  // flag set after we're done setting up server threads
409  final AtomicBoolean online = new AtomicBoolean(false);
410
411  // zookeeper connection and watcher
412  protected final ZKWatcher zooKeeper;
413
414  // master address tracker
415  private final MasterAddressTracker masterAddressTracker;
416
417  // Cluster Status Tracker
418  protected final ClusterStatusTracker clusterStatusTracker;
419
420  // Log Splitting Worker
421  private SplitLogWorker splitLogWorker;
422
423  // A sleeper that sleeps for msgInterval.
424  protected final Sleeper sleeper;
425
426  private final int shortOperationTimeout;
427
428  private final RegionServerAccounting regionServerAccounting;
429
430  private SlowLogTableOpsChore slowLogTableOpsChore = null;
431
432  // Block cache
433  private BlockCache blockCache;
434  // The cache for mob files
435  private MobFileCache mobFileCache;
436
437  /** The health check chore. */
438  private HealthCheckChore healthCheckChore;
439
440  /** The Executor status collect chore. */
441  private ExecutorStatusChore executorStatusChore;
442
443  /** The nonce manager chore. */
444  private ScheduledChore nonceManagerChore;
445
446  private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
447
448  /**
449   * The server name the Master sees us as.  Its made from the hostname the
450   * master passes us, port, and server startcode. Gets set after registration
451   * against Master.
452   */
453  protected ServerName serverName;
454
455  /**
456   * hostname specified by hostname config
457   */
458  protected String useThisHostnameInstead;
459
460  /**
461   * HBASE-18226: This config and hbase.regionserver.hostname are mutually exclusive.
462   * Exception will be thrown if both are used.
463   */
464  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
465  final static String RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY =
466    "hbase.regionserver.hostname.disable.master.reversedns";
467
468  /**
469   * This servers startcode.
470   */
471  protected final long startcode;
472
473  /**
474   * Unique identifier for the cluster we are a part of.
475   */
476  protected String clusterId;
477
478  /**
479   * Chore to clean periodically the moved region list
480   */
481  private MovedRegionsCleaner movedRegionsCleaner;
482
483  // chore for refreshing store files for secondary regions
484  private StorefileRefresherChore storefileRefresher;
485
486  private RegionServerCoprocessorHost rsHost;
487
488  private RegionServerProcedureManagerHost rspmHost;
489
490  private RegionServerRpcQuotaManager rsQuotaManager;
491  private RegionServerSpaceQuotaManager rsSpaceQuotaManager;
492
493  /**
494   * Nonce manager. Nonces are used to make operations like increment and append idempotent
495   * in the case where client doesn't receive the response from a successful operation and
496   * retries. We track the successful ops for some time via a nonce sent by client and handle
497   * duplicate operations (currently, by failing them; in future we might use MVCC to return
498   * result). Nonces are also recovered from WAL during, recovery; however, the caveats (from
499   * HBASE-3787) are:
500   * - WAL recovery is optimized, and under high load we won't read nearly nonce-timeout worth
501   *   of past records. If we don't read the records, we don't read and recover the nonces.
502   *   Some WALs within nonce-timeout at recovery may not even be present due to rolling/cleanup.
503   * - There's no WAL recovery during normal region move, so nonces will not be transfered.
504   * We can have separate additional "Nonce WAL". It will just contain bunch of numbers and
505   * won't be flushed on main path - because WAL itself also contains nonces, if we only flush
506   * it before memstore flush, for a given nonce we will either see it in the WAL (if it was
507   * never flushed to disk, it will be part of recovery), or we'll see it as part of the nonce
508   * log (or both occasionally, which doesn't matter). Nonce log file can be deleted after the
509   * latest nonce in it expired. It can also be recovered during move.
510   */
511  final ServerNonceManager nonceManager;
512
513  private UserProvider userProvider;
514
515  protected final RSRpcServices rpcServices;
516
517  private CoordinatedStateManager csm;
518
519  /**
520   * Configuration manager is used to register/deregister and notify the configuration observers
521   * when the regionserver is notified that there was a change in the on disk configs.
522   */
523  protected final ConfigurationManager configurationManager;
524
525  @VisibleForTesting
526  CompactedHFilesDischarger compactedFileDischarger;
527
528  private volatile ThroughputController flushThroughputController;
529
530  private SecureBulkLoadManager secureBulkLoadManager;
531
532  private FileSystemUtilizationChore fsUtilizationChore;
533
534  private final NettyEventLoopGroupConfig eventLoopGroupConfig;
535
536  /**
537   * Provide online slow log responses from ringbuffer
538   */
539  private NamedQueueRecorder namedQueueRecorder = null;
540
541  /**
542   * True if this RegionServer is coming up in a cluster where there is no Master;
543   * means it needs to just come up and make do without a Master to talk to: e.g. in test or
544   * HRegionServer is doing other than its usual duties: e.g. as an hollowed-out host whose only
545   * purpose is as a Replication-stream sink; see HBASE-18846 for more.
546   * TODO: can this replace {@link #TEST_SKIP_REPORTING_TRANSITION} ?
547   */
548  private final boolean masterless;
549  private static final String MASTERLESS_CONFIG_NAME = "hbase.masterless";
550
551  /**regionserver codec list **/
552  private static final String REGIONSERVER_CODEC = "hbase.regionserver.codecs";
553
554  // A timer to shutdown the process if abort takes too long
555  private Timer abortMonitor;
556
557  /**
558   * Starts a HRegionServer at the default location.
559   * <p/>
560   * Don't start any services or managers in here in the Constructor.
561   * Defer till after we register with the Master as much as possible. See {@link #startServices}.
562   */
563  public HRegionServer(final Configuration conf) throws IOException {
564    super("RegionServer");  // thread name
565    TraceUtil.initTracer(conf);
566    try {
567      this.startcode = System.currentTimeMillis();
568      this.conf = conf;
569      this.dataFsOk = true;
570      this.masterless = conf.getBoolean(MASTERLESS_CONFIG_NAME, false);
571      this.eventLoopGroupConfig = setupNetty(this.conf);
572      MemorySizeUtil.checkForClusterFreeHeapMemoryLimit(this.conf);
573      HFile.checkHFileVersion(this.conf);
574      checkCodecs(this.conf);
575      this.userProvider = UserProvider.instantiate(conf);
576      FSUtils.setupShortCircuitRead(this.conf);
577
578      // Disable usage of meta replicas in the regionserver
579      this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
580      // Config'ed params
581      this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
582      this.compactionCheckFrequency = conf.getInt(PERIOD_COMPACTION, this.threadWakeFrequency);
583      this.flushCheckFrequency = conf.getInt(PERIOD_FLUSH, this.threadWakeFrequency);
584      this.msgInterval = conf.getInt("hbase.regionserver.msginterval", 3 * 1000);
585
586      this.sleeper = new Sleeper(this.msgInterval, this);
587
588      boolean isNoncesEnabled = conf.getBoolean(HConstants.HBASE_RS_NONCES_ENABLED, true);
589      this.nonceManager = isNoncesEnabled ? new ServerNonceManager(this.conf) : null;
590
591      this.shortOperationTimeout = conf.getInt(HConstants.HBASE_RPC_SHORTOPERATION_TIMEOUT_KEY,
592          HConstants.DEFAULT_HBASE_RPC_SHORTOPERATION_TIMEOUT);
593
594      this.abortRequested = new AtomicBoolean(false);
595      this.stopped = false;
596
597      if (!(this instanceof HMaster)) {
598        final boolean isOnlineLogProviderEnabled = conf.getBoolean(
599          HConstants.SLOW_LOG_BUFFER_ENABLED_KEY,
600          HConstants.DEFAULT_ONLINE_LOG_PROVIDER_ENABLED);
601        if (isOnlineLogProviderEnabled) {
602          this.namedQueueRecorder = NamedQueueRecorder.getInstance(this.conf);
603        }
604      }
605      rpcServices = createRpcServices();
606      useThisHostnameInstead = getUseThisHostnameInstead(conf);
607      String hostName =
608          StringUtils.isBlank(useThisHostnameInstead) ? this.rpcServices.isa.getHostName()
609              : this.useThisHostnameInstead;
610      serverName = ServerName.valueOf(hostName, this.rpcServices.isa.getPort(), this.startcode);
611
612      // login the zookeeper client principal (if using security)
613      ZKUtil.loginClient(this.conf, HConstants.ZK_CLIENT_KEYTAB_FILE,
614          HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL, hostName);
615      // login the server principal (if using secure Hadoop)
616      login(userProvider, hostName);
617      // init superusers and add the server principal (if using security)
618      // or process owner as default super user.
619      Superusers.initialize(conf);
620      regionServerAccounting = new RegionServerAccounting(conf);
621
622      boolean isMasterNotCarryTable =
623          this instanceof HMaster && !LoadBalancer.isTablesOnMaster(conf);
624
625      // no need to instantiate block cache and mob file cache when master not carry table
626      if (!isMasterNotCarryTable) {
627        blockCache = BlockCacheFactory.createBlockCache(conf);
628        mobFileCache = new MobFileCache(conf);
629      }
630
631      uncaughtExceptionHandler =
632        (t, e) -> abort("Uncaught exception in executorService thread " + t.getName(), e);
633
634      initializeFileSystem();
635      spanReceiverHost = SpanReceiverHost.getInstance(getConfiguration());
636
637      this.configurationManager = new ConfigurationManager();
638      setupWindows(getConfiguration(), getConfigurationManager());
639
640      // Some unit tests don't need a cluster, so no zookeeper at all
641      if (!conf.getBoolean("hbase.testing.nocluster", false)) {
642        // Open connection to zookeeper and set primary watcher
643        zooKeeper = new ZKWatcher(conf, getProcessName() + ":" +
644          rpcServices.isa.getPort(), this, canCreateBaseZNode());
645        // If no master in cluster, skip trying to track one or look for a cluster status.
646        if (!this.masterless) {
647          if (conf.getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK,
648            DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK)) {
649            this.csm = new ZkCoordinatedStateManager(this);
650          }
651
652          masterAddressTracker = new MasterAddressTracker(getZooKeeper(), this);
653          masterAddressTracker.start();
654
655          clusterStatusTracker = new ClusterStatusTracker(zooKeeper, this);
656          clusterStatusTracker.start();
657        } else {
658          masterAddressTracker = null;
659          clusterStatusTracker = null;
660        }
661      } else {
662        zooKeeper = null;
663        masterAddressTracker = null;
664        clusterStatusTracker = null;
665      }
666      this.rpcServices.start(zooKeeper);
667      // This violates 'no starting stuff in Constructor' but Master depends on the below chore
668      // and executor being created and takes a different startup route. Lots of overlap between HRS
669      // and M (An M IS A HRS now). Need to refactor so less duplication between M and its super
670      // Master expects Constructor to put up web servers. Ugh.
671      // class HRS. TODO.
672      this.choreService = new ChoreService(getName(), true);
673      this.executorService = new ExecutorService(getName());
674      putUpWebUI();
675    } catch (Throwable t) {
676      // Make sure we log the exception. HRegionServer is often started via reflection and the
677      // cause of failed startup is lost.
678      LOG.error("Failed construction RegionServer", t);
679      throw t;
680    }
681  }
682
683  // HMaster should override this method to load the specific config for master
684  protected String getUseThisHostnameInstead(Configuration conf) throws IOException {
685    String hostname = conf.get(RS_HOSTNAME_KEY);
686    if (conf.getBoolean(RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY, false)) {
687      if (!StringUtils.isBlank(hostname)) {
688        String msg = RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY + " and " + RS_HOSTNAME_KEY +
689          " are mutually exclusive. Do not set " + RS_HOSTNAME_DISABLE_MASTER_REVERSEDNS_KEY +
690          " to true while " + RS_HOSTNAME_KEY + " is used";
691        throw new IOException(msg);
692      } else {
693        return rpcServices.isa.getHostName();
694      }
695    } else {
696      return hostname;
697    }
698  }
699
700  /**
701   * If running on Windows, do windows-specific setup.
702   */
703  private static void setupWindows(final Configuration conf, ConfigurationManager cm) {
704    if (!SystemUtils.IS_OS_WINDOWS) {
705      Signal.handle(new Signal("HUP"), signal -> {
706        conf.reloadConfiguration();
707        cm.notifyAllObservers(conf);
708      });
709    }
710  }
711
712  private static NettyEventLoopGroupConfig setupNetty(Configuration conf) {
713    // Initialize netty event loop group at start as we may use it for rpc server, rpc client & WAL.
714    NettyEventLoopGroupConfig nelgc =
715      new NettyEventLoopGroupConfig(conf, "RS-EventLoopGroup");
716    NettyRpcClientConfigHelper.setEventLoopConfig(conf, nelgc.group(), nelgc.clientChannelClass());
717    NettyAsyncFSWALConfigHelper.setEventLoopConfig(conf, nelgc.group(), nelgc.clientChannelClass());
718    return nelgc;
719  }
720
721  private void initializeFileSystem() throws IOException {
722    // Get fs instance used by this RS. Do we use checksum verification in the hbase? If hbase
723    // checksum verification enabled, then automatically switch off hdfs checksum verification.
724    boolean useHBaseChecksum = conf.getBoolean(HConstants.HBASE_CHECKSUM_VERIFICATION, true);
725    CommonFSUtils.setFsDefault(this.conf, CommonFSUtils.getWALRootDir(this.conf));
726    this.walFs = new HFileSystem(this.conf, useHBaseChecksum);
727    this.walRootDir = CommonFSUtils.getWALRootDir(this.conf);
728    // Set 'fs.defaultFS' to match the filesystem on hbase.rootdir else
729    // underlying hadoop hdfs accessors will be going against wrong filesystem
730    // (unless all is set to defaults).
731    CommonFSUtils.setFsDefault(this.conf, CommonFSUtils.getRootDir(this.conf));
732    this.dataFs = new HFileSystem(this.conf, useHBaseChecksum);
733    this.dataRootDir = CommonFSUtils.getRootDir(this.conf);
734    this.tableDescriptors =
735      new FSTableDescriptors(this.dataFs, this.dataRootDir, !canUpdateTableDescriptor(), false);
736  }
737
738  protected void login(UserProvider user, String host) throws IOException {
739    user.login(SecurityConstants.REGIONSERVER_KRB_KEYTAB_FILE,
740      SecurityConstants.REGIONSERVER_KRB_PRINCIPAL, host);
741  }
742
743  /**
744   * Wait for an active Master.
745   * See override in Master superclass for how it is used.
746   */
747  protected void waitForMasterActive() {}
748
749  protected String getProcessName() {
750    return REGIONSERVER;
751  }
752
753  protected boolean canCreateBaseZNode() {
754    return this.masterless;
755  }
756
757  protected boolean canUpdateTableDescriptor() {
758    return false;
759  }
760
761  protected RSRpcServices createRpcServices() throws IOException {
762    return new RSRpcServices(this);
763  }
764
765  protected void configureInfoServer() {
766    infoServer.addUnprivilegedServlet("rs-status", "/rs-status", RSStatusServlet.class);
767    infoServer.setAttribute(REGIONSERVER, this);
768  }
769
770  protected Class<? extends HttpServlet> getDumpServlet() {
771    return RSDumpServlet.class;
772  }
773
774  @Override
775  public boolean registerService(Service instance) {
776    // No stacking of instances is allowed for a single executorService name
777    ServiceDescriptor serviceDesc = instance.getDescriptorForType();
778    String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
779    if (coprocessorServiceHandlers.containsKey(serviceName)) {
780      LOG.error("Coprocessor executorService " + serviceName +
781        " already registered, rejecting request from " + instance);
782      return false;
783    }
784
785    coprocessorServiceHandlers.put(serviceName, instance);
786    if (LOG.isDebugEnabled()) {
787      LOG.debug(
788        "Registered regionserver coprocessor executorService: executorService=" + serviceName);
789    }
790    return true;
791  }
792
793  private Configuration cleanupConfiguration() {
794    Configuration conf = this.conf;
795    // We use ZKConnectionRegistry for all the internal communication, primarily for these reasons:
796    // - Decouples RS and master life cycles. RegionServers can continue be up independent of
797    //   masters' availability.
798    // - Configuration management for region servers (cluster internal) is much simpler when adding
799    //   new masters or removing existing masters, since only clients' config needs to be updated.
800    // - We need to retain ZKConnectionRegistry for replication use anyway, so we just extend it for
801    //   other internal connections too.
802    conf.set(HConstants.CLIENT_CONNECTION_REGISTRY_IMPL_CONF_KEY,
803        HConstants.ZK_CONNECTION_REGISTRY_CLASS);
804    if (conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM) != null) {
805      // Use server ZK cluster for server-issued connections, so we clone
806      // the conf and unset the client ZK related properties
807      conf = new Configuration(this.conf);
808      conf.unset(HConstants.CLIENT_ZOOKEEPER_QUORUM);
809    }
810    return conf;
811  }
812
813  /**
814   * Run test on configured codecs to make sure supporting libs are in place.
815   */
816  private static void checkCodecs(final Configuration c) throws IOException {
817    // check to see if the codec list is available:
818    String [] codecs = c.getStrings(REGIONSERVER_CODEC, (String[])null);
819    if (codecs == null) return;
820    for (String codec : codecs) {
821      if (!CompressionTest.testCompression(codec)) {
822        throw new IOException("Compression codec " + codec +
823          " not supported, aborting RS construction");
824      }
825    }
826  }
827
828  public String getClusterId() {
829    return this.clusterId;
830  }
831
832  /**
833   * Setup our cluster connection if not already initialized.
834   */
835  protected final synchronized void setupClusterConnection() throws IOException {
836    if (asyncClusterConnection == null) {
837      Configuration conf = cleanupConfiguration();
838      InetSocketAddress localAddress = new InetSocketAddress(this.rpcServices.isa.getAddress(), 0);
839      User user = userProvider.getCurrent();
840      asyncClusterConnection =
841        ClusterConnectionFactory.createAsyncClusterConnection(conf, localAddress, user);
842    }
843  }
844
845  /**
846   * All initialization needed before we go register with Master.<br>
847   * Do bare minimum. Do bulk of initializations AFTER we've connected to the Master.<br>
848   * In here we just put up the RpcServer, setup Connection, and ZooKeeper.
849   */
850  private void preRegistrationInitialization() {
851    try {
852      initializeZooKeeper();
853      setupClusterConnection();
854      // Setup RPC client for master communication
855      this.rpcClient = asyncClusterConnection.getRpcClient();
856    } catch (Throwable t) {
857      // Call stop if error or process will stick around for ever since server
858      // puts up non-daemon threads.
859      this.rpcServices.stop();
860      abort("Initialization of RS failed.  Hence aborting RS.", t);
861    }
862  }
863
864  /**
865   * Bring up connection to zk ensemble and then wait until a master for this cluster and then after
866   * that, wait until cluster 'up' flag has been set. This is the order in which master does things.
867   * <p>
868   * Finally open long-living server short-circuit connection.
869   */
870  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="RV_RETURN_VALUE_IGNORED_BAD_PRACTICE",
871    justification="cluster Id znode read would give us correct response")
872  private void initializeZooKeeper() throws IOException, InterruptedException {
873    // Nothing to do in here if no Master in the mix.
874    if (this.masterless) {
875      return;
876    }
877
878    // Create the master address tracker, register with zk, and start it.  Then
879    // block until a master is available.  No point in starting up if no master
880    // running.
881    blockAndCheckIfStopped(this.masterAddressTracker);
882
883    // Wait on cluster being up.  Master will set this flag up in zookeeper
884    // when ready.
885    blockAndCheckIfStopped(this.clusterStatusTracker);
886
887    // If we are HMaster then the cluster id should have already been set.
888    if (clusterId == null) {
889      // Retrieve clusterId
890      // Since cluster status is now up
891      // ID should have already been set by HMaster
892      try {
893        clusterId = ZKClusterId.readClusterIdZNode(this.zooKeeper);
894        if (clusterId == null) {
895          this.abort("Cluster ID has not been set");
896        }
897        LOG.info("ClusterId : " + clusterId);
898      } catch (KeeperException e) {
899        this.abort("Failed to retrieve Cluster ID", e);
900      }
901    }
902
903    waitForMasterActive();
904    if (isStopped() || isAborted()) {
905      return; // No need for further initialization
906    }
907
908    // watch for snapshots and other procedures
909    try {
910      rspmHost = new RegionServerProcedureManagerHost();
911      rspmHost.loadProcedures(conf);
912      rspmHost.initialize(this);
913    } catch (KeeperException e) {
914      this.abort("Failed to reach coordination cluster when creating procedure handler.", e);
915    }
916  }
917
918  /**
919   * Utilty method to wait indefinitely on a znode availability while checking
920   * if the region server is shut down
921   * @param tracker znode tracker to use
922   * @throws IOException any IO exception, plus if the RS is stopped
923   * @throws InterruptedException if the waiting thread is interrupted
924   */
925  private void blockAndCheckIfStopped(ZKNodeTracker tracker)
926      throws IOException, InterruptedException {
927    while (tracker.blockUntilAvailable(this.msgInterval, false) == null) {
928      if (this.stopped) {
929        throw new IOException("Received the shutdown message while waiting.");
930      }
931    }
932  }
933
934  /**
935   * @return True if the cluster is up.
936   */
937  @Override
938  public boolean isClusterUp() {
939    return this.masterless ||
940        (this.clusterStatusTracker != null && this.clusterStatusTracker.isClusterUp());
941  }
942
943  /**
944   * The HRegionServer sticks in this loop until closed.
945   */
946  @Override
947  public void run() {
948    if (isStopped()) {
949      LOG.info("Skipping run; stopped");
950      return;
951    }
952    try {
953      // Do pre-registration initializations; zookeeper, lease threads, etc.
954      preRegistrationInitialization();
955    } catch (Throwable e) {
956      abort("Fatal exception during initialization", e);
957    }
958
959    try {
960      if (!isStopped() && !isAborted()) {
961        ShutdownHook.install(conf, dataFs, this, Thread.currentThread());
962        // Initialize the RegionServerCoprocessorHost now that our ephemeral
963        // node was created, in case any coprocessors want to use ZooKeeper
964        this.rsHost = new RegionServerCoprocessorHost(this, this.conf);
965
966        // Try and register with the Master; tell it we are here.  Break if server is stopped or
967        // the clusterup flag is down or hdfs went wacky. Once registered successfully, go ahead and
968        // start up all Services. Use RetryCounter to get backoff in case Master is struggling to
969        // come up.
970        LOG.debug("About to register with Master.");
971        RetryCounterFactory rcf =
972          new RetryCounterFactory(Integer.MAX_VALUE, this.sleeper.getPeriod(), 1000 * 60 * 5);
973        RetryCounter rc = rcf.create();
974        while (keepLooping()) {
975          RegionServerStartupResponse w = reportForDuty();
976          if (w == null) {
977            long sleepTime = rc.getBackoffTimeAndIncrementAttempts();
978            LOG.warn("reportForDuty failed; sleeping {} ms and then retrying.", sleepTime);
979            this.sleeper.sleep(sleepTime);
980          } else {
981            handleReportForDutyResponse(w);
982            break;
983          }
984        }
985      }
986
987      if (!isStopped() && isHealthy()) {
988        // start the snapshot handler and other procedure handlers,
989        // since the server is ready to run
990        if (this.rspmHost != null) {
991          this.rspmHost.start();
992        }
993        // Start the Quota Manager
994        if (this.rsQuotaManager != null) {
995          rsQuotaManager.start(getRpcServer().getScheduler());
996        }
997        if (this.rsSpaceQuotaManager != null) {
998          this.rsSpaceQuotaManager.start();
999        }
1000      }
1001
1002      // We registered with the Master.  Go into run mode.
1003      long lastMsg = System.currentTimeMillis();
1004      long oldRequestCount = -1;
1005      // The main run loop.
1006      while (!isStopped() && isHealthy()) {
1007        if (!isClusterUp()) {
1008          if (onlineRegions.isEmpty()) {
1009            stop("Exiting; cluster shutdown set and not carrying any regions");
1010          } else if (!this.stopping) {
1011            this.stopping = true;
1012            LOG.info("Closing user regions");
1013            closeUserRegions(this.abortRequested.get());
1014          } else {
1015            boolean allUserRegionsOffline = areAllUserRegionsOffline();
1016            if (allUserRegionsOffline) {
1017              // Set stopped if no more write requests tp meta tables
1018              // since last time we went around the loop.  Any open
1019              // meta regions will be closed on our way out.
1020              if (oldRequestCount == getWriteRequestCount()) {
1021                stop("Stopped; only catalog regions remaining online");
1022                break;
1023              }
1024              oldRequestCount = getWriteRequestCount();
1025            } else {
1026              // Make sure all regions have been closed -- some regions may
1027              // have not got it because we were splitting at the time of
1028              // the call to closeUserRegions.
1029              closeUserRegions(this.abortRequested.get());
1030            }
1031            LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
1032          }
1033        }
1034        long now = System.currentTimeMillis();
1035        if ((now - lastMsg) >= msgInterval) {
1036          tryRegionServerReport(lastMsg, now);
1037          lastMsg = System.currentTimeMillis();
1038        }
1039        if (!isStopped() && !isAborted()) {
1040          this.sleeper.sleep();
1041        }
1042      } // for
1043    } catch (Throwable t) {
1044      if (!rpcServices.checkOOME(t)) {
1045        String prefix = t instanceof YouAreDeadException? "": "Unhandled: ";
1046        abort(prefix + t.getMessage(), t);
1047      }
1048    }
1049
1050    if (this.leaseManager != null) {
1051      this.leaseManager.closeAfterLeasesExpire();
1052    }
1053    if (this.splitLogWorker != null) {
1054      splitLogWorker.stop();
1055    }
1056    if (this.infoServer != null) {
1057      LOG.info("Stopping infoServer");
1058      try {
1059        this.infoServer.stop();
1060      } catch (Exception e) {
1061        LOG.error("Failed to stop infoServer", e);
1062      }
1063    }
1064    // Send cache a shutdown.
1065    if (blockCache != null) {
1066      blockCache.shutdown();
1067    }
1068    if (mobFileCache != null) {
1069      mobFileCache.shutdown();
1070    }
1071
1072    if (movedRegionsCleaner != null) {
1073      movedRegionsCleaner.stop("Region Server stopping");
1074    }
1075
1076    // Send interrupts to wake up threads if sleeping so they notice shutdown.
1077    // TODO: Should we check they are alive? If OOME could have exited already
1078    if (this.hMemManager != null) this.hMemManager.stop();
1079    if (this.cacheFlusher != null) this.cacheFlusher.interruptIfNecessary();
1080    if (this.compactSplitThread != null) this.compactSplitThread.interruptIfNecessary();
1081
1082    // Stop the snapshot and other procedure handlers, forcefully killing all running tasks
1083    if (rspmHost != null) {
1084      rspmHost.stop(this.abortRequested.get() || this.killed);
1085    }
1086
1087    if (this.killed) {
1088      // Just skip out w/o closing regions.  Used when testing.
1089    } else if (abortRequested.get()) {
1090      if (this.dataFsOk) {
1091        closeUserRegions(abortRequested.get()); // Don't leave any open file handles
1092      }
1093      LOG.info("aborting server " + this.serverName);
1094    } else {
1095      closeUserRegions(abortRequested.get());
1096      LOG.info("stopping server " + this.serverName);
1097    }
1098
1099    if (this.asyncClusterConnection != null) {
1100      try {
1101        this.asyncClusterConnection.close();
1102      } catch (IOException e) {
1103        // Although the {@link Closeable} interface throws an {@link
1104        // IOException}, in reality, the implementation would never do that.
1105        LOG.warn("Attempt to close server's AsyncClusterConnection failed.", e);
1106      }
1107    }
1108    // Closing the compactSplit thread before closing meta regions
1109    if (!this.killed && containsMetaTableRegions()) {
1110      if (!abortRequested.get() || this.dataFsOk) {
1111        if (this.compactSplitThread != null) {
1112          this.compactSplitThread.join();
1113          this.compactSplitThread = null;
1114        }
1115        closeMetaTableRegions(abortRequested.get());
1116      }
1117    }
1118
1119    if (!this.killed && this.dataFsOk) {
1120      waitOnAllRegionsToClose(abortRequested.get());
1121      LOG.info("stopping server " + this.serverName + "; all regions closed.");
1122    }
1123
1124    // Stop the quota manager
1125    if (rsQuotaManager != null) {
1126      rsQuotaManager.stop();
1127    }
1128    if (rsSpaceQuotaManager != null) {
1129      rsSpaceQuotaManager.stop();
1130      rsSpaceQuotaManager = null;
1131    }
1132
1133    // flag may be changed when closing regions throws exception.
1134    if (this.dataFsOk) {
1135      shutdownWAL(!abortRequested.get());
1136    }
1137
1138    // Make sure the proxy is down.
1139    if (this.rssStub != null) {
1140      this.rssStub = null;
1141    }
1142    if (this.lockStub != null) {
1143      this.lockStub = null;
1144    }
1145    if (this.rpcClient != null) {
1146      this.rpcClient.close();
1147    }
1148    if (this.leaseManager != null) {
1149      this.leaseManager.close();
1150    }
1151    if (this.pauseMonitor != null) {
1152      this.pauseMonitor.stop();
1153    }
1154
1155    if (!killed) {
1156      stopServiceThreads();
1157    }
1158
1159    if (this.rpcServices != null) {
1160      this.rpcServices.stop();
1161    }
1162
1163    try {
1164      deleteMyEphemeralNode();
1165    } catch (KeeperException.NoNodeException nn) {
1166      // pass
1167    } catch (KeeperException e) {
1168      LOG.warn("Failed deleting my ephemeral node", e);
1169    }
1170    // We may have failed to delete the znode at the previous step, but
1171    //  we delete the file anyway: a second attempt to delete the znode is likely to fail again.
1172    ZNodeClearer.deleteMyEphemeralNodeOnDisk();
1173
1174    if (this.zooKeeper != null) {
1175      this.zooKeeper.close();
1176    }
1177    this.shutDown = true;
1178    LOG.info("Exiting; stopping=" + this.serverName + "; zookeeper connection closed.");
1179  }
1180
1181  private boolean containsMetaTableRegions() {
1182    return onlineRegions.containsKey(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
1183  }
1184
1185  private boolean areAllUserRegionsOffline() {
1186    if (getNumberOfOnlineRegions() > 2) return false;
1187    boolean allUserRegionsOffline = true;
1188    for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
1189      if (!e.getValue().getRegionInfo().isMetaRegion()) {
1190        allUserRegionsOffline = false;
1191        break;
1192      }
1193    }
1194    return allUserRegionsOffline;
1195  }
1196
1197  /**
1198   * @return Current write count for all online regions.
1199   */
1200  private long getWriteRequestCount() {
1201    long writeCount = 0;
1202    for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
1203      writeCount += e.getValue().getWriteRequestsCount();
1204    }
1205    return writeCount;
1206  }
1207
1208  @VisibleForTesting
1209  protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
1210      throws IOException {
1211    RegionServerStatusService.BlockingInterface rss = rssStub;
1212    if (rss == null) {
1213      // the current server could be stopping.
1214      return;
1215    }
1216    ClusterStatusProtos.ServerLoad sl = buildServerLoad(reportStartTime, reportEndTime);
1217    try {
1218      RegionServerReportRequest.Builder request = RegionServerReportRequest.newBuilder();
1219      request.setServer(ProtobufUtil.toServerName(this.serverName));
1220      request.setLoad(sl);
1221      rss.regionServerReport(null, request.build());
1222    } catch (ServiceException se) {
1223      IOException ioe = ProtobufUtil.getRemoteException(se);
1224      if (ioe instanceof YouAreDeadException) {
1225        // This will be caught and handled as a fatal error in run()
1226        throw ioe;
1227      }
1228      if (rssStub == rss) {
1229        rssStub = null;
1230      }
1231      // Couldn't connect to the master, get location from zk and reconnect
1232      // Method blocks until new master is found or we are stopped
1233      createRegionServerStatusStub(true);
1234    }
1235  }
1236
1237  /**
1238   * Reports the given map of Regions and their size on the filesystem to the active Master.
1239   *
1240   * @param regionSizeStore The store containing region sizes
1241   * @return false if FileSystemUtilizationChore should pause reporting to master. true otherwise
1242   */
1243  public boolean reportRegionSizesForQuotas(RegionSizeStore regionSizeStore) {
1244    RegionServerStatusService.BlockingInterface rss = rssStub;
1245    if (rss == null) {
1246      // the current server could be stopping.
1247      LOG.trace("Skipping Region size report to HMaster as stub is null");
1248      return true;
1249    }
1250    try {
1251      buildReportAndSend(rss, regionSizeStore);
1252    } catch (ServiceException se) {
1253      IOException ioe = ProtobufUtil.getRemoteException(se);
1254      if (ioe instanceof PleaseHoldException) {
1255        LOG.trace("Failed to report region sizes to Master because it is initializing."
1256            + " This will be retried.", ioe);
1257        // The Master is coming up. Will retry the report later. Avoid re-creating the stub.
1258        return true;
1259      }
1260      if (rssStub == rss) {
1261        rssStub = null;
1262      }
1263      createRegionServerStatusStub(true);
1264      if (ioe instanceof DoNotRetryIOException) {
1265        DoNotRetryIOException doNotRetryEx = (DoNotRetryIOException) ioe;
1266        if (doNotRetryEx.getCause() != null) {
1267          Throwable t = doNotRetryEx.getCause();
1268          if (t instanceof UnsupportedOperationException) {
1269            LOG.debug("master doesn't support ReportRegionSpaceUse, pause before retrying");
1270            return false;
1271          }
1272        }
1273      }
1274      LOG.debug("Failed to report region sizes to Master. This will be retried.", ioe);
1275    }
1276    return true;
1277  }
1278
1279  /**
1280   * Builds the region size report and sends it to the master. Upon successful sending of the
1281   * report, the region sizes that were sent are marked as sent.
1282   *
1283   * @param rss The stub to send to the Master
1284   * @param regionSizeStore The store containing region sizes
1285   */
1286  private void buildReportAndSend(RegionServerStatusService.BlockingInterface rss,
1287      RegionSizeStore regionSizeStore) throws ServiceException {
1288    RegionSpaceUseReportRequest request =
1289        buildRegionSpaceUseReportRequest(Objects.requireNonNull(regionSizeStore));
1290    rss.reportRegionSpaceUse(null, request);
1291    // Record the number of size reports sent
1292    if (metricsRegionServer != null) {
1293      metricsRegionServer.incrementNumRegionSizeReportsSent(regionSizeStore.size());
1294    }
1295  }
1296
1297  /**
1298   * Builds a {@link RegionSpaceUseReportRequest} protobuf message from the region size map.
1299   *
1300   * @param regionSizes The size in bytes of regions
1301   * @return The corresponding protocol buffer message.
1302   */
1303  RegionSpaceUseReportRequest buildRegionSpaceUseReportRequest(RegionSizeStore regionSizes) {
1304    RegionSpaceUseReportRequest.Builder request = RegionSpaceUseReportRequest.newBuilder();
1305    for (Entry<RegionInfo, RegionSize> entry : regionSizes) {
1306      request.addSpaceUse(convertRegionSize(entry.getKey(), entry.getValue().getSize()));
1307    }
1308    return request.build();
1309  }
1310
1311  /**
1312   * Converts a pair of {@link RegionInfo} and {@code long} into a {@link RegionSpaceUse}
1313   * protobuf message.
1314   *
1315   * @param regionInfo The RegionInfo
1316   * @param sizeInBytes The size in bytes of the Region
1317   * @return The protocol buffer
1318   */
1319  RegionSpaceUse convertRegionSize(RegionInfo regionInfo, Long sizeInBytes) {
1320    return RegionSpaceUse.newBuilder()
1321        .setRegionInfo(ProtobufUtil.toRegionInfo(Objects.requireNonNull(regionInfo)))
1322        .setRegionSize(Objects.requireNonNull(sizeInBytes))
1323        .build();
1324  }
1325
1326  private ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime)
1327      throws IOException {
1328    // We're getting the MetricsRegionServerWrapper here because the wrapper computes requests
1329    // per second, and other metrics  As long as metrics are part of ServerLoad it's best to use
1330    // the wrapper to compute those numbers in one place.
1331    // In the long term most of these should be moved off of ServerLoad and the heart beat.
1332    // Instead they should be stored in an HBase table so that external visibility into HBase is
1333    // improved; Additionally the load balancer will be able to take advantage of a more complete
1334    // history.
1335    MetricsRegionServerWrapper regionServerWrapper = metricsRegionServer.getRegionServerWrapper();
1336    Collection<HRegion> regions = getOnlineRegionsLocalContext();
1337    long usedMemory = -1L;
1338    long maxMemory = -1L;
1339    final MemoryUsage usage = MemorySizeUtil.safeGetHeapMemoryUsage();
1340    if (usage != null) {
1341      usedMemory = usage.getUsed();
1342      maxMemory = usage.getMax();
1343    }
1344
1345    ClusterStatusProtos.ServerLoad.Builder serverLoad = ClusterStatusProtos.ServerLoad.newBuilder();
1346    serverLoad.setNumberOfRequests((int) regionServerWrapper.getRequestsPerSecond());
1347    serverLoad.setTotalNumberOfRequests(regionServerWrapper.getTotalRequestCount());
1348    serverLoad.setUsedHeapMB((int)(usedMemory / 1024 / 1024));
1349    serverLoad.setMaxHeapMB((int) (maxMemory / 1024 / 1024));
1350    Set<String> coprocessors = getWAL(null).getCoprocessorHost().getCoprocessors();
1351    Builder coprocessorBuilder = Coprocessor.newBuilder();
1352    for (String coprocessor : coprocessors) {
1353      serverLoad.addCoprocessors(coprocessorBuilder.setName(coprocessor).build());
1354    }
1355    RegionLoad.Builder regionLoadBldr = RegionLoad.newBuilder();
1356    RegionSpecifier.Builder regionSpecifier = RegionSpecifier.newBuilder();
1357    for (HRegion region : regions) {
1358      if (region.getCoprocessorHost() != null) {
1359        Set<String> regionCoprocessors = region.getCoprocessorHost().getCoprocessors();
1360        for (String regionCoprocessor : regionCoprocessors) {
1361          serverLoad.addCoprocessors(coprocessorBuilder.setName(regionCoprocessor).build());
1362        }
1363      }
1364      serverLoad.addRegionLoads(createRegionLoad(region, regionLoadBldr, regionSpecifier));
1365      for (String coprocessor : getWAL(region.getRegionInfo()).getCoprocessorHost()
1366          .getCoprocessors()) {
1367        serverLoad.addCoprocessors(coprocessorBuilder.setName(coprocessor).build());
1368      }
1369    }
1370    serverLoad.setReportStartTime(reportStartTime);
1371    serverLoad.setReportEndTime(reportEndTime);
1372    if (this.infoServer != null) {
1373      serverLoad.setInfoServerPort(this.infoServer.getPort());
1374    } else {
1375      serverLoad.setInfoServerPort(-1);
1376    }
1377    MetricsUserAggregateSource userSource =
1378        metricsRegionServer.getMetricsUserAggregate().getSource();
1379    if (userSource != null) {
1380      Map<String, MetricsUserSource> userMetricMap = userSource.getUserSources();
1381      for (Entry<String, MetricsUserSource> entry : userMetricMap.entrySet()) {
1382        serverLoad.addUserLoads(createUserLoad(entry.getKey(), entry.getValue()));
1383      }
1384    }
1385    // for the replicationLoad purpose. Only need to get from one executorService
1386    // either source or sink will get the same info
1387    ReplicationSourceService rsources = getReplicationSourceService();
1388
1389    if (rsources != null) {
1390      // always refresh first to get the latest value
1391      ReplicationLoad rLoad = rsources.refreshAndGetReplicationLoad();
1392      if (rLoad != null) {
1393        serverLoad.setReplLoadSink(rLoad.getReplicationLoadSink());
1394        for (ClusterStatusProtos.ReplicationLoadSource rLS :
1395            rLoad.getReplicationLoadSourceEntries()) {
1396          serverLoad.addReplLoadSource(rLS);
1397        }
1398
1399      }
1400    }
1401
1402    return serverLoad.build();
1403  }
1404
1405  private String getOnlineRegionsAsPrintableString() {
1406    StringBuilder sb = new StringBuilder();
1407    for (Region r: this.onlineRegions.values()) {
1408      if (sb.length() > 0) sb.append(", ");
1409      sb.append(r.getRegionInfo().getEncodedName());
1410    }
1411    return sb.toString();
1412  }
1413
1414  /**
1415   * Wait on regions close.
1416   */
1417  private void waitOnAllRegionsToClose(final boolean abort) {
1418    // Wait till all regions are closed before going out.
1419    int lastCount = -1;
1420    long previousLogTime = 0;
1421    Set<String> closedRegions = new HashSet<>();
1422    boolean interrupted = false;
1423    try {
1424      while (!onlineRegions.isEmpty()) {
1425        int count = getNumberOfOnlineRegions();
1426        // Only print a message if the count of regions has changed.
1427        if (count != lastCount) {
1428          // Log every second at most
1429          if (System.currentTimeMillis() > (previousLogTime + 1000)) {
1430            previousLogTime = System.currentTimeMillis();
1431            lastCount = count;
1432            LOG.info("Waiting on " + count + " regions to close");
1433            // Only print out regions still closing if a small number else will
1434            // swamp the log.
1435            if (count < 10 && LOG.isDebugEnabled()) {
1436              LOG.debug("Online Regions=" + this.onlineRegions);
1437            }
1438          }
1439        }
1440        // Ensure all user regions have been sent a close. Use this to
1441        // protect against the case where an open comes in after we start the
1442        // iterator of onlineRegions to close all user regions.
1443        for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
1444          RegionInfo hri = e.getValue().getRegionInfo();
1445          if (!this.regionsInTransitionInRS.containsKey(hri.getEncodedNameAsBytes()) &&
1446              !closedRegions.contains(hri.getEncodedName())) {
1447            closedRegions.add(hri.getEncodedName());
1448            // Don't update zk with this close transition; pass false.
1449            closeRegionIgnoreErrors(hri, abort);
1450          }
1451        }
1452        // No regions in RIT, we could stop waiting now.
1453        if (this.regionsInTransitionInRS.isEmpty()) {
1454          if (!onlineRegions.isEmpty()) {
1455            LOG.info("We were exiting though online regions are not empty," +
1456                " because some regions failed closing");
1457          }
1458          break;
1459        } else {
1460          LOG.debug("Waiting on {}", this.regionsInTransitionInRS.keySet().stream().
1461            map(e -> Bytes.toString(e)).collect(Collectors.joining(", ")));
1462        }
1463        if (sleepInterrupted(200)) {
1464          interrupted = true;
1465        }
1466      }
1467    } finally {
1468      if (interrupted) {
1469        Thread.currentThread().interrupt();
1470      }
1471    }
1472  }
1473
1474  private static boolean sleepInterrupted(long millis) {
1475    boolean interrupted = false;
1476    try {
1477      Thread.sleep(millis);
1478    } catch (InterruptedException e) {
1479      LOG.warn("Interrupted while sleeping");
1480      interrupted = true;
1481    }
1482    return interrupted;
1483  }
1484
1485  private void shutdownWAL(final boolean close) {
1486    if (this.walFactory != null) {
1487      try {
1488        if (close) {
1489          walFactory.close();
1490        } else {
1491          walFactory.shutdown();
1492        }
1493      } catch (Throwable e) {
1494        e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;
1495        LOG.error("Shutdown / close of WAL failed: " + e);
1496        LOG.debug("Shutdown / close exception details:", e);
1497      }
1498    }
1499  }
1500
1501  /**
1502   * get NamedQueue Provider to add different logs to ringbuffer
1503   *
1504   * @return NamedQueueRecorder
1505   */
1506  public NamedQueueRecorder getNamedQueueRecorder() {
1507    return this.namedQueueRecorder;
1508  }
1509
1510  /*
1511   * Run init. Sets up wal and starts up all server threads.
1512   *
1513   * @param c Extra configuration.
1514   */
1515  protected void handleReportForDutyResponse(final RegionServerStartupResponse c)
1516  throws IOException {
1517    try {
1518      boolean updateRootDir = false;
1519      for (NameStringPair e : c.getMapEntriesList()) {
1520        String key = e.getName();
1521        // The hostname the master sees us as.
1522        if (key.equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
1523          String hostnameFromMasterPOV = e.getValue();
1524          this.serverName = ServerName.valueOf(hostnameFromMasterPOV, rpcServices.isa.getPort(),
1525              this.startcode);
1526          if (!StringUtils.isBlank(useThisHostnameInstead) &&
1527              !hostnameFromMasterPOV.equals(useThisHostnameInstead)) {
1528            String msg = "Master passed us a different hostname to use; was=" +
1529                this.useThisHostnameInstead + ", but now=" + hostnameFromMasterPOV;
1530            LOG.error(msg);
1531            throw new IOException(msg);
1532          }
1533          if (StringUtils.isBlank(useThisHostnameInstead) &&
1534              !hostnameFromMasterPOV.equals(rpcServices.isa.getHostName())) {
1535            String msg = "Master passed us a different hostname to use; was=" +
1536                rpcServices.isa.getHostName() + ", but now=" + hostnameFromMasterPOV;
1537            LOG.error(msg);
1538          }
1539          continue;
1540        }
1541
1542        String value = e.getValue();
1543        if (key.equals(HConstants.HBASE_DIR)) {
1544          if (value != null && !value.equals(conf.get(HConstants.HBASE_DIR))) {
1545            updateRootDir = true;
1546          }
1547        }
1548
1549        if (LOG.isDebugEnabled()) {
1550          LOG.debug("Config from master: " + key + "=" + value);
1551        }
1552        this.conf.set(key, value);
1553      }
1554      // Set our ephemeral znode up in zookeeper now we have a name.
1555      createMyEphemeralNode();
1556
1557      if (updateRootDir) {
1558        // initialize file system by the config fs.defaultFS and hbase.rootdir from master
1559        initializeFileSystem();
1560      }
1561
1562      // hack! Maps DFSClient => RegionServer for logs.  HDFS made this
1563      // config param for task trackers, but we can piggyback off of it.
1564      if (this.conf.get("mapreduce.task.attempt.id") == null) {
1565        this.conf.set("mapreduce.task.attempt.id", "hb_rs_" + this.serverName.toString());
1566      }
1567
1568      // Save it in a file, this will allow to see if we crash
1569      ZNodeClearer.writeMyEphemeralNodeOnDisk(getMyEphemeralNodePath());
1570
1571      // This call sets up an initialized replication and WAL. Later we start it up.
1572      setupWALAndReplication();
1573      // Init in here rather than in constructor after thread name has been set
1574      final MetricsTable metricsTable =
1575          new MetricsTable(new MetricsTableWrapperAggregateImpl(this));
1576      this.metricsRegionServerImpl = new MetricsRegionServerWrapperImpl(this);
1577      this.metricsRegionServer = new MetricsRegionServer(
1578          metricsRegionServerImpl, conf, metricsTable);
1579      // Now that we have a metrics source, start the pause monitor
1580      this.pauseMonitor = new JvmPauseMonitor(conf, getMetrics().getMetricsSource());
1581      pauseMonitor.start();
1582
1583      // There is a rare case where we do NOT want services to start. Check config.
1584      if (getConfiguration().getBoolean("hbase.regionserver.workers", true)) {
1585        startServices();
1586      }
1587      // In here we start up the replication Service. Above we initialized it. TODO. Reconcile.
1588      // or make sense of it.
1589      startReplicationService();
1590
1591
1592      // Set up ZK
1593      LOG.info("Serving as " + this.serverName + ", RpcServer on " + rpcServices.isa +
1594          ", sessionid=0x" +
1595          Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
1596
1597      // Wake up anyone waiting for this server to online
1598      synchronized (online) {
1599        online.set(true);
1600        online.notifyAll();
1601      }
1602    } catch (Throwable e) {
1603      stop("Failed initialization");
1604      throw convertThrowableToIOE(cleanup(e, "Failed init"),
1605          "Region server startup failed");
1606    } finally {
1607      sleeper.skipSleepCycle();
1608    }
1609  }
1610
1611  protected void initializeMemStoreChunkCreator() {
1612    if (MemStoreLAB.isEnabled(conf)) {
1613      // MSLAB is enabled. So initialize MemStoreChunkPool
1614      // By this time, the MemstoreFlusher is already initialized. We can get the global limits from
1615      // it.
1616      Pair<Long, MemoryType> pair = MemorySizeUtil.getGlobalMemStoreSize(conf);
1617      long globalMemStoreSize = pair.getFirst();
1618      boolean offheap = this.regionServerAccounting.isOffheap();
1619      // When off heap memstore in use, take full area for chunk pool.
1620      float poolSizePercentage = offheap? 1.0F:
1621          conf.getFloat(MemStoreLAB.CHUNK_POOL_MAXSIZE_KEY, MemStoreLAB.POOL_MAX_SIZE_DEFAULT);
1622      float initialCountPercentage = conf.getFloat(MemStoreLAB.CHUNK_POOL_INITIALSIZE_KEY,
1623          MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT);
1624      int chunkSize = conf.getInt(MemStoreLAB.CHUNK_SIZE_KEY, MemStoreLAB.CHUNK_SIZE_DEFAULT);
1625      // init the chunkCreator
1626      ChunkCreator.initialize(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
1627        initialCountPercentage, this.hMemManager);
1628    }
1629  }
1630
1631  private void startHeapMemoryManager() {
1632    if (this.blockCache != null) {
1633      this.hMemManager =
1634          new HeapMemoryManager(this.blockCache, this.cacheFlusher, this, regionServerAccounting);
1635      this.hMemManager.start(getChoreService());
1636    }
1637  }
1638
1639  private void createMyEphemeralNode() throws KeeperException {
1640    RegionServerInfo.Builder rsInfo = RegionServerInfo.newBuilder();
1641    rsInfo.setInfoPort(infoServer != null ? infoServer.getPort() : -1);
1642    rsInfo.setVersionInfo(ProtobufUtil.getVersionInfo());
1643    byte[] data = ProtobufUtil.prependPBMagic(rsInfo.build().toByteArray());
1644    ZKUtil.createEphemeralNodeAndWatch(this.zooKeeper, getMyEphemeralNodePath(), data);
1645  }
1646
1647  private void deleteMyEphemeralNode() throws KeeperException {
1648    ZKUtil.deleteNode(this.zooKeeper, getMyEphemeralNodePath());
1649  }
1650
1651  @Override
1652  public RegionServerAccounting getRegionServerAccounting() {
1653    return regionServerAccounting;
1654  }
1655
1656  /**
1657   * @param r Region to get RegionLoad for.
1658   * @param regionLoadBldr the RegionLoad.Builder, can be null
1659   * @param regionSpecifier the RegionSpecifier.Builder, can be null
1660   * @return RegionLoad instance.
1661   */
1662  RegionLoad createRegionLoad(final HRegion r, RegionLoad.Builder regionLoadBldr,
1663      RegionSpecifier.Builder regionSpecifier) throws IOException {
1664    byte[] name = r.getRegionInfo().getRegionName();
1665    int stores = 0;
1666    int storefiles = 0;
1667    int storeRefCount = 0;
1668    int maxCompactedStoreFileRefCount = 0;
1669    int storeUncompressedSizeMB = 0;
1670    int storefileSizeMB = 0;
1671    int memstoreSizeMB = (int) (r.getMemStoreDataSize() / 1024 / 1024);
1672    long storefileIndexSizeKB = 0;
1673    int rootLevelIndexSizeKB = 0;
1674    int totalStaticIndexSizeKB = 0;
1675    int totalStaticBloomSizeKB = 0;
1676    long totalCompactingKVs = 0;
1677    long currentCompactedKVs = 0;
1678    List<HStore> storeList = r.getStores();
1679    stores += storeList.size();
1680    for (HStore store : storeList) {
1681      storefiles += store.getStorefilesCount();
1682      int currentStoreRefCount = store.getStoreRefCount();
1683      storeRefCount += currentStoreRefCount;
1684      int currentMaxCompactedStoreFileRefCount = store.getMaxCompactedStoreFileRefCount();
1685      maxCompactedStoreFileRefCount = Math.max(maxCompactedStoreFileRefCount,
1686        currentMaxCompactedStoreFileRefCount);
1687      storeUncompressedSizeMB += (int) (store.getStoreSizeUncompressed() / 1024 / 1024);
1688      storefileSizeMB += (int) (store.getStorefilesSize() / 1024 / 1024);
1689      //TODO: storefileIndexSizeKB is same with rootLevelIndexSizeKB?
1690      storefileIndexSizeKB += store.getStorefilesRootLevelIndexSize() / 1024;
1691      CompactionProgress progress = store.getCompactionProgress();
1692      if (progress != null) {
1693        totalCompactingKVs += progress.getTotalCompactingKVs();
1694        currentCompactedKVs += progress.currentCompactedKVs;
1695      }
1696      rootLevelIndexSizeKB += (int) (store.getStorefilesRootLevelIndexSize() / 1024);
1697      totalStaticIndexSizeKB += (int) (store.getTotalStaticIndexSize() / 1024);
1698      totalStaticBloomSizeKB += (int) (store.getTotalStaticBloomSize() / 1024);
1699    }
1700
1701    HDFSBlocksDistribution hdfsBd = r.getHDFSBlocksDistribution();
1702    float dataLocality = hdfsBd.getBlockLocalityIndex(serverName.getHostname());
1703    float dataLocalityForSsd = hdfsBd.getBlockLocalityIndexForSsd(serverName.getHostname());
1704    long blocksTotalWeight = hdfsBd.getUniqueBlocksTotalWeight();
1705    long blocksLocalWeight = hdfsBd.getBlocksLocalWeight(serverName.getHostname());
1706    long blocksLocalWithSsdWeight = hdfsBd.getBlocksLocalWithSsdWeight(serverName.getHostname());
1707    if (regionLoadBldr == null) {
1708      regionLoadBldr = RegionLoad.newBuilder();
1709    }
1710    if (regionSpecifier == null) {
1711      regionSpecifier = RegionSpecifier.newBuilder();
1712    }
1713    regionSpecifier.setType(RegionSpecifierType.REGION_NAME);
1714    regionSpecifier.setValue(UnsafeByteOperations.unsafeWrap(name));
1715    regionLoadBldr.setRegionSpecifier(regionSpecifier.build())
1716      .setStores(stores)
1717      .setStorefiles(storefiles)
1718      .setStoreRefCount(storeRefCount)
1719      .setMaxCompactedStoreFileRefCount(maxCompactedStoreFileRefCount)
1720      .setStoreUncompressedSizeMB(storeUncompressedSizeMB)
1721      .setStorefileSizeMB(storefileSizeMB)
1722      .setMemStoreSizeMB(memstoreSizeMB)
1723      .setStorefileIndexSizeKB(storefileIndexSizeKB)
1724      .setRootIndexSizeKB(rootLevelIndexSizeKB)
1725      .setTotalStaticIndexSizeKB(totalStaticIndexSizeKB)
1726      .setTotalStaticBloomSizeKB(totalStaticBloomSizeKB)
1727      .setReadRequestsCount(r.getReadRequestsCount())
1728      .setCpRequestsCount(r.getCpRequestsCount())
1729      .setFilteredReadRequestsCount(r.getFilteredReadRequestsCount())
1730      .setWriteRequestsCount(r.getWriteRequestsCount())
1731      .setTotalCompactingKVs(totalCompactingKVs)
1732      .setCurrentCompactedKVs(currentCompactedKVs)
1733      .setDataLocality(dataLocality)
1734      .setDataLocalityForSsd(dataLocalityForSsd)
1735      .setBlocksLocalWeight(blocksLocalWeight)
1736      .setBlocksLocalWithSsdWeight(blocksLocalWithSsdWeight)
1737      .setBlocksTotalWeight(blocksTotalWeight)
1738      .setLastMajorCompactionTs(r.getOldestHfileTs(true));
1739    r.setCompleteSequenceId(regionLoadBldr);
1740
1741    return regionLoadBldr.build();
1742  }
1743
1744  private UserLoad createUserLoad(String user, MetricsUserSource userSource) {
1745    UserLoad.Builder userLoadBldr = UserLoad.newBuilder();
1746    userLoadBldr.setUserName(user);
1747    userSource.getClientMetrics().values().stream().map(
1748      clientMetrics -> ClusterStatusProtos.ClientMetrics.newBuilder()
1749            .setHostName(clientMetrics.getHostName())
1750            .setWriteRequestsCount(clientMetrics.getWriteRequestsCount())
1751            .setFilteredRequestsCount(clientMetrics.getFilteredReadRequests())
1752            .setReadRequestsCount(clientMetrics.getReadRequestsCount()).build())
1753        .forEach(userLoadBldr::addClientMetrics);
1754    return userLoadBldr.build();
1755  }
1756
1757  public RegionLoad createRegionLoad(final String encodedRegionName) throws IOException {
1758    HRegion r = onlineRegions.get(encodedRegionName);
1759    return r != null ? createRegionLoad(r, null, null) : null;
1760  }
1761
1762  /**
1763   * Inner class that runs on a long period checking if regions need compaction.
1764   */
1765  private static class CompactionChecker extends ScheduledChore {
1766    private final HRegionServer instance;
1767    private final int majorCompactPriority;
1768    private final static int DEFAULT_PRIORITY = Integer.MAX_VALUE;
1769    //Iteration is 1-based rather than 0-based so we don't check for compaction
1770    // immediately upon region server startup
1771    private long iteration = 1;
1772
1773    CompactionChecker(final HRegionServer h, final int sleepTime, final Stoppable stopper) {
1774      super("CompactionChecker", stopper, sleepTime);
1775      this.instance = h;
1776      LOG.info(this.getName() + " runs every " + Duration.ofMillis(sleepTime));
1777
1778      /* MajorCompactPriority is configurable.
1779       * If not set, the compaction will use default priority.
1780       */
1781      this.majorCompactPriority = this.instance.conf.
1782          getInt("hbase.regionserver.compactionChecker.majorCompactPriority",
1783              DEFAULT_PRIORITY);
1784    }
1785
1786    @Override
1787    protected void chore() {
1788      for (Region r : this.instance.onlineRegions.values()) {
1789        // Skip compaction if region is read only
1790        if (r == null || r.isReadOnly()) {
1791          continue;
1792        }
1793
1794        HRegion hr = (HRegion) r;
1795        for (HStore s : hr.stores.values()) {
1796          try {
1797            long multiplier = s.getCompactionCheckMultiplier();
1798            assert multiplier > 0;
1799            if (iteration % multiplier != 0) {
1800              continue;
1801            }
1802            if (s.needsCompaction()) {
1803              // Queue a compaction. Will recognize if major is needed.
1804              this.instance.compactSplitThread.requestSystemCompaction(hr, s,
1805                getName() + " requests compaction");
1806            } else if (s.shouldPerformMajorCompaction()) {
1807              s.triggerMajorCompaction();
1808              if (majorCompactPriority == DEFAULT_PRIORITY ||
1809                  majorCompactPriority > hr.getCompactPriority()) {
1810                this.instance.compactSplitThread.requestCompaction(hr, s,
1811                    getName() + " requests major compaction; use default priority",
1812                    Store.NO_PRIORITY,
1813                CompactionLifeCycleTracker.DUMMY, null);
1814              } else {
1815                this.instance.compactSplitThread.requestCompaction(hr, s,
1816                    getName() + " requests major compaction; use configured priority",
1817                    this.majorCompactPriority, CompactionLifeCycleTracker.DUMMY, null);
1818              }
1819            }
1820          } catch (IOException e) {
1821            LOG.warn("Failed major compaction check on " + r, e);
1822          }
1823        }
1824      }
1825      iteration = (iteration == Long.MAX_VALUE) ? 0 : (iteration + 1);
1826    }
1827  }
1828
1829  private static class PeriodicMemStoreFlusher extends ScheduledChore {
1830    private final HRegionServer server;
1831    private final static int RANGE_OF_DELAY = 5 * 60; // 5 min in seconds
1832    private final static int MIN_DELAY_TIME = 0; // millisec
1833    private final long rangeOfDelayMs;
1834
1835    PeriodicMemStoreFlusher(int cacheFlushInterval, final HRegionServer server) {
1836      super("MemstoreFlusherChore", server, cacheFlushInterval);
1837      this.server = server;
1838
1839      final long configuredRangeOfDelay = server.getConfiguration().getInt(
1840          "hbase.regionserver.periodicmemstoreflusher.rangeofdelayseconds", RANGE_OF_DELAY);
1841      this.rangeOfDelayMs = TimeUnit.SECONDS.toMillis(configuredRangeOfDelay);
1842    }
1843
1844    @Override
1845    protected void chore() {
1846      final StringBuilder whyFlush = new StringBuilder();
1847      for (HRegion r : this.server.onlineRegions.values()) {
1848        if (r == null) continue;
1849        if (r.shouldFlush(whyFlush)) {
1850          FlushRequester requester = server.getFlushRequester();
1851          if (requester != null) {
1852            long randomDelay = RandomUtils.nextLong(0, rangeOfDelayMs) + MIN_DELAY_TIME;
1853            //Throttle the flushes by putting a delay. If we don't throttle, and there
1854            //is a balanced write-load on the regions in a table, we might end up
1855            //overwhelming the filesystem with too many flushes at once.
1856            if (requester.requestDelayedFlush(r, randomDelay)) {
1857              LOG.info("{} requesting flush of {} because {} after random delay {} ms",
1858                  getName(), r.getRegionInfo().getRegionNameAsString(),  whyFlush.toString(),
1859                  randomDelay);
1860            }
1861          }
1862        }
1863      }
1864    }
1865  }
1866
1867  /**
1868   * Report the status of the server. A server is online once all the startup is
1869   * completed (setting up filesystem, starting executorService threads, etc.). This
1870   * method is designed mostly to be useful in tests.
1871   *
1872   * @return true if online, false if not.
1873   */
1874  public boolean isOnline() {
1875    return online.get();
1876  }
1877
1878  /**
1879   * Setup WAL log and replication if enabled. Replication setup is done in here because it wants to
1880   * be hooked up to WAL.
1881   */
1882  private void setupWALAndReplication() throws IOException {
1883    boolean isMasterNoTableOrSystemTableOnly = this instanceof HMaster &&
1884        !LoadBalancer.isMasterCanHostUserRegions(conf);
1885    WALFactory factory =
1886        new WALFactory(conf, serverName.toString(), !isMasterNoTableOrSystemTableOnly);
1887    if (!isMasterNoTableOrSystemTableOnly) {
1888      // TODO Replication make assumptions here based on the default filesystem impl
1889      Path oldLogDir = new Path(walRootDir, HConstants.HREGION_OLDLOGDIR_NAME);
1890      String logName = AbstractFSWALProvider.getWALDirectoryName(this.serverName.toString());
1891
1892      Path logDir = new Path(walRootDir, logName);
1893      LOG.debug("logDir={}", logDir);
1894      if (this.walFs.exists(logDir)) {
1895        throw new RegionServerRunningException(
1896            "Region server has already created directory at " + this.serverName.toString());
1897      }
1898      // Always create wal directory as now we need this when master restarts to find out the live
1899      // region servers.
1900      if (!this.walFs.mkdirs(logDir)) {
1901        throw new IOException("Can not create wal directory " + logDir);
1902      }
1903      // Instantiate replication if replication enabled. Pass it the log directories.
1904      createNewReplicationInstance(conf, this, this.walFs, logDir, oldLogDir,
1905        factory.getWALProvider());
1906    }
1907    this.walFactory = factory;
1908  }
1909
1910  /**
1911   * Start up replication source and sink handlers.
1912   */
1913  private void startReplicationService() throws IOException {
1914    if (this.replicationSourceHandler == this.replicationSinkHandler &&
1915        this.replicationSourceHandler != null) {
1916      this.replicationSourceHandler.startReplicationService();
1917    } else {
1918      if (this.replicationSourceHandler != null) {
1919        this.replicationSourceHandler.startReplicationService();
1920      }
1921      if (this.replicationSinkHandler != null) {
1922        this.replicationSinkHandler.startReplicationService();
1923      }
1924    }
1925  }
1926
1927  /**
1928   * @return Master address tracker instance.
1929   */
1930  public MasterAddressTracker getMasterAddressTracker() {
1931    return this.masterAddressTracker;
1932  }
1933
1934  /**
1935   * Start maintenance Threads, Server, Worker and lease checker threads.
1936   * Start all threads we need to run. This is called after we've successfully
1937   * registered with the Master.
1938   * Install an UncaughtExceptionHandler that calls abort of RegionServer if we
1939   * get an unhandled exception. We cannot set the handler on all threads.
1940   * Server's internal Listener thread is off limits. For Server, if an OOME, it
1941   * waits a while then retries. Meantime, a flush or a compaction that tries to
1942   * run should trigger same critical condition and the shutdown will run. On
1943   * its way out, this server will shut down Server. Leases are sort of
1944   * inbetween. It has an internal thread that while it inherits from Chore, it
1945   * keeps its own internal stop mechanism so needs to be stopped by this
1946   * hosting server. Worker logs the exception and exits.
1947   */
1948  private void startServices() throws IOException {
1949    if (!isStopped() && !isAborted()) {
1950      initializeThreads();
1951    }
1952    this.secureBulkLoadManager = new SecureBulkLoadManager(this.conf, asyncClusterConnection);
1953    this.secureBulkLoadManager.start();
1954
1955    // Health checker thread.
1956    if (isHealthCheckerConfigured()) {
1957      int sleepTime = this.conf.getInt(HConstants.HEALTH_CHORE_WAKE_FREQ,
1958      HConstants.DEFAULT_THREAD_WAKE_FREQUENCY);
1959      healthCheckChore = new HealthCheckChore(sleepTime, this, getConfiguration());
1960    }
1961    // Executor status collect thread.
1962    if (this.conf.getBoolean(HConstants.EXECUTOR_STATUS_COLLECT_ENABLED,
1963        HConstants.DEFAULT_EXECUTOR_STATUS_COLLECT_ENABLED)) {
1964      int sleepTime = this.conf.getInt(ExecutorStatusChore.WAKE_FREQ,
1965          ExecutorStatusChore.DEFAULT_WAKE_FREQ);
1966      executorStatusChore = new ExecutorStatusChore(sleepTime, this, this.getExecutorService(),
1967          this.metricsRegionServer.getMetricsSource());
1968    }
1969
1970    this.walRoller = new LogRoller(this);
1971    this.flushThroughputController = FlushThroughputControllerFactory.create(this, conf);
1972    this.procedureResultReporter = new RemoteProcedureResultReporter(this);
1973
1974    // Create the CompactedFileDischarger chore executorService. This chore helps to
1975    // remove the compacted files that will no longer be used in reads.
1976    // Default is 2 mins. The default value for TTLCleaner is 5 mins so we set this to
1977    // 2 mins so that compacted files can be archived before the TTLCleaner runs
1978    int cleanerInterval =
1979      conf.getInt("hbase.hfile.compaction.discharger.interval", 2 * 60 * 1000);
1980    this.compactedFileDischarger =
1981      new CompactedHFilesDischarger(cleanerInterval, this, this);
1982    choreService.scheduleChore(compactedFileDischarger);
1983
1984    // Start executor services
1985    this.executorService.startExecutorService(ExecutorType.RS_OPEN_REGION,
1986        conf.getInt("hbase.regionserver.executor.openregion.threads", 3));
1987    this.executorService.startExecutorService(ExecutorType.RS_OPEN_META,
1988        conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
1989    this.executorService.startExecutorService(ExecutorType.RS_OPEN_PRIORITY_REGION,
1990        conf.getInt("hbase.regionserver.executor.openpriorityregion.threads", 3));
1991    this.executorService.startExecutorService(ExecutorType.RS_CLOSE_REGION,
1992        conf.getInt("hbase.regionserver.executor.closeregion.threads", 3));
1993    this.executorService.startExecutorService(ExecutorType.RS_CLOSE_META,
1994        conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
1995    if (conf.getBoolean(StoreScanner.STORESCANNER_PARALLEL_SEEK_ENABLE, false)) {
1996      this.executorService.startExecutorService(ExecutorType.RS_PARALLEL_SEEK,
1997          conf.getInt("hbase.storescanner.parallel.seek.threads", 10));
1998    }
1999    this.executorService.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, conf.getInt(
2000        HBASE_SPLIT_WAL_MAX_SPLITTER, DEFAULT_HBASE_SPLIT_WAL_MAX_SPLITTER));
2001    // Start the threads for compacted files discharger
2002    this.executorService.startExecutorService(ExecutorType.RS_COMPACTED_FILES_DISCHARGER,
2003        conf.getInt(CompactionConfiguration.HBASE_HFILE_COMPACTION_DISCHARGER_THREAD_COUNT, 10));
2004    if (ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(conf)) {
2005      this.executorService.startExecutorService(ExecutorType.RS_REGION_REPLICA_FLUSH_OPS,
2006          conf.getInt("hbase.regionserver.region.replica.flusher.threads",
2007              conf.getInt("hbase.regionserver.executor.openregion.threads", 3)));
2008    }
2009    this.executorService.startExecutorService(ExecutorType.RS_REFRESH_PEER,
2010      conf.getInt("hbase.regionserver.executor.refresh.peer.threads", 2));
2011    this.executorService.startExecutorService(ExecutorType.RS_REPLAY_SYNC_REPLICATION_WAL,
2012      conf.getInt("hbase.regionserver.executor.replay.sync.replication.wal.threads", 1));
2013    this.executorService.startExecutorService(ExecutorType.RS_SWITCH_RPC_THROTTLE,
2014      conf.getInt("hbase.regionserver.executor.switch.rpc.throttle.threads", 1));
2015
2016    Threads.setDaemonThreadRunning(this.walRoller, getName() + ".logRoller",
2017        uncaughtExceptionHandler);
2018    if (this.cacheFlusher != null) {
2019      this.cacheFlusher.start(uncaughtExceptionHandler);
2020    }
2021    Threads.setDaemonThreadRunning(this.procedureResultReporter,
2022      getName() + ".procedureResultReporter", uncaughtExceptionHandler);
2023
2024    if (this.compactionChecker != null) {
2025      choreService.scheduleChore(compactionChecker);
2026    }
2027    if (this.periodicFlusher != null) {
2028      choreService.scheduleChore(periodicFlusher);
2029    }
2030    if (this.healthCheckChore != null) {
2031      choreService.scheduleChore(healthCheckChore);
2032    }
2033    if (this.executorStatusChore != null) {
2034      choreService.scheduleChore(executorStatusChore);
2035    }
2036    if (this.nonceManagerChore != null) {
2037      choreService.scheduleChore(nonceManagerChore);
2038    }
2039    if (this.storefileRefresher != null) {
2040      choreService.scheduleChore(storefileRefresher);
2041    }
2042    if (this.movedRegionsCleaner != null) {
2043      choreService.scheduleChore(movedRegionsCleaner);
2044    }
2045    if (this.fsUtilizationChore != null) {
2046      choreService.scheduleChore(fsUtilizationChore);
2047    }
2048    if (this.slowLogTableOpsChore != null) {
2049      choreService.scheduleChore(slowLogTableOpsChore);
2050    }
2051
2052    // Leases is not a Thread. Internally it runs a daemon thread. If it gets
2053    // an unhandled exception, it will just exit.
2054    Threads.setDaemonThreadRunning(this.leaseManager, getName() + ".leaseChecker",
2055        uncaughtExceptionHandler);
2056
2057    // Create the log splitting worker and start it
2058    // set a smaller retries to fast fail otherwise splitlogworker could be blocked for
2059    // quite a while inside Connection layer. The worker won't be available for other
2060    // tasks even after current task is preempted after a split task times out.
2061    Configuration sinkConf = HBaseConfiguration.create(conf);
2062    sinkConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
2063        conf.getInt("hbase.log.replay.retries.number", 8)); // 8 retries take about 23 seconds
2064    sinkConf.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
2065        conf.getInt("hbase.log.replay.rpc.timeout", 30000)); // default 30 seconds
2066    sinkConf.setInt(HConstants.HBASE_CLIENT_SERVERSIDE_RETRIES_MULTIPLIER, 1);
2067    if (this.csm != null && conf.getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK,
2068      DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK)) {
2069      // SplitLogWorker needs csm. If none, don't start this.
2070      this.splitLogWorker = new SplitLogWorker(sinkConf, this, this, walFactory);
2071      splitLogWorker.start();
2072      LOG.debug("SplitLogWorker started");
2073    }
2074
2075    // Memstore services.
2076    startHeapMemoryManager();
2077    // Call it after starting HeapMemoryManager.
2078    initializeMemStoreChunkCreator();
2079  }
2080
2081  private void initializeThreads() {
2082    // Cache flushing thread.
2083    this.cacheFlusher = new MemStoreFlusher(conf, this);
2084
2085    // Compaction thread
2086    this.compactSplitThread = new CompactSplit(this);
2087
2088    // Background thread to check for compactions; needed if region has not gotten updates
2089    // in a while. It will take care of not checking too frequently on store-by-store basis.
2090    this.compactionChecker = new CompactionChecker(this, this.compactionCheckFrequency, this);
2091    this.periodicFlusher = new PeriodicMemStoreFlusher(this.flushCheckFrequency, this);
2092    this.leaseManager = new LeaseManager(this.threadWakeFrequency);
2093
2094    final boolean isSlowLogTableEnabled = conf.getBoolean(HConstants.SLOW_LOG_SYS_TABLE_ENABLED_KEY,
2095      HConstants.DEFAULT_SLOW_LOG_SYS_TABLE_ENABLED_KEY);
2096    if (isSlowLogTableEnabled) {
2097      // default chore duration: 10 min
2098      final int duration = conf.getInt("hbase.slowlog.systable.chore.duration", 10 * 60 * 1000);
2099      slowLogTableOpsChore = new SlowLogTableOpsChore(this, duration, this.namedQueueRecorder);
2100    }
2101
2102    // Create the thread to clean the moved regions list
2103    movedRegionsCleaner = MovedRegionsCleaner.create(this);
2104
2105    if (this.nonceManager != null) {
2106      // Create the scheduled chore that cleans up nonces.
2107      nonceManagerChore = this.nonceManager.createCleanupScheduledChore(this);
2108    }
2109
2110    // Setup the Quota Manager
2111    rsQuotaManager = new RegionServerRpcQuotaManager(this);
2112    rsSpaceQuotaManager = new RegionServerSpaceQuotaManager(this);
2113
2114    if (QuotaUtil.isQuotaEnabled(conf)) {
2115      this.fsUtilizationChore = new FileSystemUtilizationChore(this);
2116    }
2117
2118
2119    boolean onlyMetaRefresh = false;
2120    int storefileRefreshPeriod = conf.getInt(
2121        StorefileRefresherChore.REGIONSERVER_STOREFILE_REFRESH_PERIOD,
2122        StorefileRefresherChore.DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD);
2123    if (storefileRefreshPeriod == 0) {
2124      storefileRefreshPeriod = conf.getInt(
2125          StorefileRefresherChore.REGIONSERVER_META_STOREFILE_REFRESH_PERIOD,
2126          StorefileRefresherChore.DEFAULT_REGIONSERVER_STOREFILE_REFRESH_PERIOD);
2127      onlyMetaRefresh = true;
2128    }
2129    if (storefileRefreshPeriod > 0) {
2130      this.storefileRefresher = new StorefileRefresherChore(storefileRefreshPeriod,
2131          onlyMetaRefresh, this, this);
2132    }
2133    registerConfigurationObservers();
2134  }
2135
2136  private void registerConfigurationObservers() {
2137    // Registering the compactSplitThread object with the ConfigurationManager.
2138    configurationManager.registerObserver(this.compactSplitThread);
2139    configurationManager.registerObserver(this.rpcServices);
2140    configurationManager.registerObserver(this);
2141  }
2142
2143  /**
2144   * Puts up the webui.
2145   */
2146  private void putUpWebUI() throws IOException {
2147    int port = this.conf.getInt(HConstants.REGIONSERVER_INFO_PORT,
2148      HConstants.DEFAULT_REGIONSERVER_INFOPORT);
2149    String addr = this.conf.get("hbase.regionserver.info.bindAddress", "0.0.0.0");
2150
2151    if(this instanceof HMaster) {
2152      port = conf.getInt(HConstants.MASTER_INFO_PORT,
2153          HConstants.DEFAULT_MASTER_INFOPORT);
2154      addr = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
2155    }
2156    // -1 is for disabling info server
2157    if (port < 0) {
2158      return;
2159    }
2160
2161    if (!Addressing.isLocalAddress(InetAddress.getByName(addr))) {
2162      String msg =
2163          "Failed to start http info server. Address " + addr
2164              + " does not belong to this host. Correct configuration parameter: "
2165              + "hbase.regionserver.info.bindAddress";
2166      LOG.error(msg);
2167      throw new IOException(msg);
2168    }
2169    // check if auto port bind enabled
2170    boolean auto = this.conf.getBoolean(HConstants.REGIONSERVER_INFO_PORT_AUTO, false);
2171    while (true) {
2172      try {
2173        this.infoServer = new InfoServer(getProcessName(), addr, port, false, this.conf);
2174        infoServer.addPrivilegedServlet("dump", "/dump", getDumpServlet());
2175        configureInfoServer();
2176        this.infoServer.start();
2177        break;
2178      } catch (BindException e) {
2179        if (!auto) {
2180          // auto bind disabled throw BindException
2181          LOG.error("Failed binding http info server to port: " + port);
2182          throw e;
2183        }
2184        // auto bind enabled, try to use another port
2185        LOG.info("Failed binding http info server to port: " + port);
2186        port++;
2187      }
2188    }
2189    port = this.infoServer.getPort();
2190    conf.setInt(HConstants.REGIONSERVER_INFO_PORT, port);
2191    int masterInfoPort = conf.getInt(HConstants.MASTER_INFO_PORT,
2192      HConstants.DEFAULT_MASTER_INFOPORT);
2193    conf.setInt("hbase.master.info.port.orig", masterInfoPort);
2194    conf.setInt(HConstants.MASTER_INFO_PORT, port);
2195  }
2196
2197  /*
2198   * Verify that server is healthy
2199   */
2200  private boolean isHealthy() {
2201    if (!dataFsOk) {
2202      // File system problem
2203      return false;
2204    }
2205    // Verify that all threads are alive
2206    boolean healthy = (this.leaseManager == null || this.leaseManager.isAlive())
2207        && (this.cacheFlusher == null || this.cacheFlusher.isAlive())
2208        && (this.walRoller == null || this.walRoller.isAlive())
2209        && (this.compactionChecker == null || this.compactionChecker.isScheduled())
2210        && (this.periodicFlusher == null || this.periodicFlusher.isScheduled());
2211    if (!healthy) {
2212      stop("One or more threads are no longer alive -- stop");
2213    }
2214    return healthy;
2215  }
2216
2217  @Override
2218  public List<WAL> getWALs() {
2219    return walFactory.getWALs();
2220  }
2221
2222  @Override
2223  public WAL getWAL(RegionInfo regionInfo) throws IOException {
2224    WAL wal = walFactory.getWAL(regionInfo);
2225    if (this.walRoller != null) {
2226      this.walRoller.addWAL(wal);
2227    }
2228    return wal;
2229  }
2230
2231  public LogRoller getWalRoller() {
2232    return walRoller;
2233  }
2234
2235  WALFactory getWalFactory() {
2236    return walFactory;
2237  }
2238
2239  @Override
2240  public Connection getConnection() {
2241    return getAsyncConnection().toConnection();
2242  }
2243
2244  @Override
2245  public void stop(final String msg) {
2246    stop(msg, false, RpcServer.getRequestUser().orElse(null));
2247  }
2248
2249  /**
2250   * Stops the regionserver.
2251   * @param msg Status message
2252   * @param force True if this is a regionserver abort
2253   * @param user The user executing the stop request, or null if no user is associated
2254   */
2255  public void stop(final String msg, final boolean force, final User user) {
2256    if (!this.stopped) {
2257      LOG.info("***** STOPPING region server '" + this + "' *****");
2258      if (this.rsHost != null) {
2259        // when forced via abort don't allow CPs to override
2260        try {
2261          this.rsHost.preStop(msg, user);
2262        } catch (IOException ioe) {
2263          if (!force) {
2264            LOG.warn("The region server did not stop", ioe);
2265            return;
2266          }
2267          LOG.warn("Skipping coprocessor exception on preStop() due to forced shutdown", ioe);
2268        }
2269      }
2270      this.stopped = true;
2271      LOG.info("STOPPED: " + msg);
2272      // Wakes run() if it is sleeping
2273      sleeper.skipSleepCycle();
2274    }
2275  }
2276
2277  public void waitForServerOnline(){
2278    while (!isStopped() && !isOnline()) {
2279      synchronized (online) {
2280        try {
2281          online.wait(msgInterval);
2282        } catch (InterruptedException ie) {
2283          Thread.currentThread().interrupt();
2284          break;
2285        }
2286      }
2287    }
2288  }
2289
2290  @Override
2291  public void postOpenDeployTasks(final PostOpenDeployContext context) throws IOException {
2292    HRegion r = context.getRegion();
2293    long openProcId = context.getOpenProcId();
2294    long masterSystemTime = context.getMasterSystemTime();
2295    rpcServices.checkOpen();
2296    LOG.info("Post open deploy tasks for {}, pid={}, masterSystemTime={}",
2297      r.getRegionInfo().getRegionNameAsString(), openProcId, masterSystemTime);
2298    // Do checks to see if we need to compact (references or too many files)
2299    // Skip compaction check if region is read only
2300    if (!r.isReadOnly()) {
2301      for (HStore s : r.stores.values()) {
2302        if (s.hasReferences() || s.needsCompaction()) {
2303          this.compactSplitThread.requestSystemCompaction(r, s, "Opening Region");
2304        }
2305      }
2306    }
2307    long openSeqNum = r.getOpenSeqNum();
2308    if (openSeqNum == HConstants.NO_SEQNUM) {
2309      // If we opened a region, we should have read some sequence number from it.
2310      LOG.error(
2311        "No sequence number found when opening " + r.getRegionInfo().getRegionNameAsString());
2312      openSeqNum = 0;
2313    }
2314
2315    // Notify master
2316    if (!reportRegionStateTransition(new RegionStateTransitionContext(TransitionCode.OPENED,
2317      openSeqNum, openProcId, masterSystemTime, r.getRegionInfo()))) {
2318      throw new IOException(
2319        "Failed to report opened region to master: " + r.getRegionInfo().getRegionNameAsString());
2320    }
2321
2322    triggerFlushInPrimaryRegion(r);
2323
2324    LOG.debug("Finished post open deploy task for " + r.getRegionInfo().getRegionNameAsString());
2325  }
2326
2327  /**
2328   * Helper method for use in tests. Skip the region transition report when there's no master
2329   * around to receive it.
2330   */
2331  private boolean skipReportingTransition(final RegionStateTransitionContext context) {
2332    final TransitionCode code = context.getCode();
2333    final long openSeqNum = context.getOpenSeqNum();
2334    long masterSystemTime = context.getMasterSystemTime();
2335    final RegionInfo[] hris = context.getHris();
2336
2337    if (code == TransitionCode.OPENED) {
2338      Preconditions.checkArgument(hris != null && hris.length == 1);
2339      if (hris[0].isMetaRegion()) {
2340        try {
2341          MetaTableLocator.setMetaLocation(getZooKeeper(), serverName,
2342              hris[0].getReplicaId(), RegionState.State.OPEN);
2343        } catch (KeeperException e) {
2344          LOG.info("Failed to update meta location", e);
2345          return false;
2346        }
2347      } else {
2348        try {
2349          MetaTableAccessor.updateRegionLocation(asyncClusterConnection.toConnection(), hris[0],
2350              serverName, openSeqNum, masterSystemTime);
2351        } catch (IOException e) {
2352          LOG.info("Failed to update meta", e);
2353          return false;
2354        }
2355      }
2356    }
2357    return true;
2358  }
2359
2360  private ReportRegionStateTransitionRequest createReportRegionStateTransitionRequest(
2361      final RegionStateTransitionContext context) {
2362    final TransitionCode code = context.getCode();
2363    final long openSeqNum = context.getOpenSeqNum();
2364    final RegionInfo[] hris = context.getHris();
2365    final long[] procIds = context.getProcIds();
2366
2367    ReportRegionStateTransitionRequest.Builder builder =
2368        ReportRegionStateTransitionRequest.newBuilder();
2369    builder.setServer(ProtobufUtil.toServerName(serverName));
2370    RegionStateTransition.Builder transition = builder.addTransitionBuilder();
2371    transition.setTransitionCode(code);
2372    if (code == TransitionCode.OPENED && openSeqNum >= 0) {
2373      transition.setOpenSeqNum(openSeqNum);
2374    }
2375    for (RegionInfo hri: hris) {
2376      transition.addRegionInfo(ProtobufUtil.toRegionInfo(hri));
2377    }
2378    for (long procId: procIds) {
2379      transition.addProcId(procId);
2380    }
2381
2382    return builder.build();
2383  }
2384
2385  @Override
2386  public boolean reportRegionStateTransition(final RegionStateTransitionContext context) {
2387    if (TEST_SKIP_REPORTING_TRANSITION) {
2388      return skipReportingTransition(context);
2389    }
2390    final ReportRegionStateTransitionRequest request =
2391        createReportRegionStateTransitionRequest(context);
2392
2393    // Time to pause if master says 'please hold'. Make configurable if needed.
2394    final long initPauseTime = 1000;
2395    int tries = 0;
2396    long pauseTime;
2397    // Keep looping till we get an error. We want to send reports even though server is going down.
2398    // Only go down if clusterConnection is null. It is set to null almost as last thing as the
2399    // HRegionServer does down.
2400    while (this.asyncClusterConnection != null && !this.asyncClusterConnection.isClosed()) {
2401      RegionServerStatusService.BlockingInterface rss = rssStub;
2402      try {
2403        if (rss == null) {
2404          createRegionServerStatusStub();
2405          continue;
2406        }
2407        ReportRegionStateTransitionResponse response =
2408          rss.reportRegionStateTransition(null, request);
2409        if (response.hasErrorMessage()) {
2410          LOG.info("TRANSITION FAILED " + request + ": " + response.getErrorMessage());
2411          break;
2412        }
2413        // Log if we had to retry else don't log unless TRACE. We want to
2414        // know if were successful after an attempt showed in logs as failed.
2415        if (tries > 0 || LOG.isTraceEnabled()) {
2416          LOG.info("TRANSITION REPORTED " + request);
2417        }
2418        // NOTE: Return mid-method!!!
2419        return true;
2420      } catch (ServiceException se) {
2421        IOException ioe = ProtobufUtil.getRemoteException(se);
2422        boolean pause =
2423            ioe instanceof ServerNotRunningYetException || ioe instanceof PleaseHoldException
2424                || ioe instanceof CallQueueTooBigException;
2425        if (pause) {
2426          // Do backoff else we flood the Master with requests.
2427          pauseTime = ConnectionUtils.getPauseTime(initPauseTime, tries);
2428        } else {
2429          pauseTime = initPauseTime; // Reset.
2430        }
2431        LOG.info("Failed report transition " +
2432          TextFormat.shortDebugString(request) + "; retry (#" + tries + ")" +
2433            (pause?
2434                " after " + pauseTime + "ms delay (Master is coming online...).":
2435                " immediately."),
2436            ioe);
2437        if (pause) Threads.sleep(pauseTime);
2438        tries++;
2439        if (rssStub == rss) {
2440          rssStub = null;
2441        }
2442      }
2443    }
2444    return false;
2445  }
2446
2447  /**
2448   * Trigger a flush in the primary region replica if this region is a secondary replica. Does not
2449   * block this thread. See RegionReplicaFlushHandler for details.
2450   */
2451  private void triggerFlushInPrimaryRegion(final HRegion region) {
2452    if (ServerRegionReplicaUtil.isDefaultReplica(region.getRegionInfo())) {
2453      return;
2454    }
2455    if (!ServerRegionReplicaUtil.isRegionReplicaReplicationEnabled(region.conf) ||
2456        !ServerRegionReplicaUtil.isRegionReplicaWaitForPrimaryFlushEnabled(
2457          region.conf)) {
2458      region.setReadsEnabled(true);
2459      return;
2460    }
2461
2462    region.setReadsEnabled(false); // disable reads before marking the region as opened.
2463    // RegionReplicaFlushHandler might reset this.
2464
2465    // submit it to be handled by one of the handlers so that we do not block OpenRegionHandler
2466    if (this.executorService != null) {
2467      this.executorService.submit(new RegionReplicaFlushHandler(this, region));
2468    }
2469  }
2470
2471  @Override
2472  public RpcServerInterface getRpcServer() {
2473    return rpcServices.rpcServer;
2474  }
2475
2476  @VisibleForTesting
2477  public RSRpcServices getRSRpcServices() {
2478    return rpcServices;
2479  }
2480
2481  /**
2482   * Cause the server to exit without closing the regions it is serving, the log
2483   * it is using and without notifying the master. Used unit testing and on
2484   * catastrophic events such as HDFS is yanked out from under hbase or we OOME.
2485   *
2486   * @param reason
2487   *          the reason we are aborting
2488   * @param cause
2489   *          the exception that caused the abort, or null
2490   */
2491  @Override
2492  public void abort(String reason, Throwable cause) {
2493    if (!setAbortRequested()) {
2494      // Abort already in progress, ignore the new request.
2495      LOG.debug(
2496          "Abort already in progress. Ignoring the current request with reason: {}", reason);
2497      return;
2498    }
2499    String msg = "***** ABORTING region server " + this + ": " + reason + " *****";
2500    if (cause != null) {
2501      LOG.error(HBaseMarkers.FATAL, msg, cause);
2502    } else {
2503      LOG.error(HBaseMarkers.FATAL, msg);
2504    }
2505    // HBASE-4014: show list of coprocessors that were loaded to help debug
2506    // regionserver crashes.Note that we're implicitly using
2507    // java.util.HashSet's toString() method to print the coprocessor names.
2508    LOG.error(HBaseMarkers.FATAL, "RegionServer abort: loaded coprocessors are: " +
2509        CoprocessorHost.getLoadedCoprocessors());
2510    // Try and dump metrics if abort -- might give clue as to how fatal came about....
2511    try {
2512      LOG.info("Dump of metrics as JSON on abort: " + DumpRegionServerMetrics.dumpMetrics());
2513    } catch (MalformedObjectNameException | IOException e) {
2514      LOG.warn("Failed dumping metrics", e);
2515    }
2516
2517    // Do our best to report our abort to the master, but this may not work
2518    try {
2519      if (cause != null) {
2520        msg += "\nCause:\n" + Throwables.getStackTraceAsString(cause);
2521      }
2522      // Report to the master but only if we have already registered with the master.
2523      RegionServerStatusService.BlockingInterface rss = rssStub;
2524      if (rss != null && this.serverName != null) {
2525        ReportRSFatalErrorRequest.Builder builder =
2526          ReportRSFatalErrorRequest.newBuilder();
2527        builder.setServer(ProtobufUtil.toServerName(this.serverName));
2528        builder.setErrorMessage(msg);
2529        rss.reportRSFatalError(null, builder.build());
2530      }
2531    } catch (Throwable t) {
2532      LOG.warn("Unable to report fatal error to master", t);
2533    }
2534
2535    scheduleAbortTimer();
2536    // shutdown should be run as the internal user
2537    stop(reason, true, null);
2538  }
2539
2540  /**
2541   * Sets the abort state if not already set.
2542   * @return True if abortRequested set to True successfully, false if an abort is already in
2543   * progress.
2544   */
2545  protected boolean setAbortRequested() {
2546    return abortRequested.compareAndSet(false, true);
2547  }
2548
2549  /**
2550   * @see HRegionServer#abort(String, Throwable)
2551   */
2552  public void abort(String reason) {
2553    abort(reason, null);
2554  }
2555
2556  @Override
2557  public boolean isAborted() {
2558    return abortRequested.get();
2559  }
2560
2561  /*
2562   * Simulate a kill -9 of this server. Exits w/o closing regions or cleaninup
2563   * logs but it does close socket in case want to bring up server on old
2564   * hostname+port immediately.
2565   */
2566  @VisibleForTesting
2567  protected void kill() {
2568    this.killed = true;
2569    abort("Simulated kill");
2570  }
2571
2572  // Limits the time spent in the shutdown process.
2573  private void scheduleAbortTimer() {
2574    if (this.abortMonitor == null) {
2575      this.abortMonitor = new Timer("Abort regionserver monitor", true);
2576      TimerTask abortTimeoutTask = null;
2577      try {
2578        Constructor<? extends TimerTask> timerTaskCtor =
2579          Class.forName(conf.get(ABORT_TIMEOUT_TASK, SystemExitWhenAbortTimeout.class.getName()))
2580            .asSubclass(TimerTask.class).getDeclaredConstructor();
2581        timerTaskCtor.setAccessible(true);
2582        abortTimeoutTask = timerTaskCtor.newInstance();
2583      } catch (Exception e) {
2584        LOG.warn("Initialize abort timeout task failed", e);
2585      }
2586      if (abortTimeoutTask != null) {
2587        abortMonitor.schedule(abortTimeoutTask, conf.getLong(ABORT_TIMEOUT, DEFAULT_ABORT_TIMEOUT));
2588      }
2589    }
2590  }
2591
2592  /**
2593   * Wait on all threads to finish. Presumption is that all closes and stops
2594   * have already been called.
2595   */
2596  protected void stopServiceThreads() {
2597    // clean up the scheduled chores
2598    if (this.choreService != null) {
2599      choreService.cancelChore(nonceManagerChore);
2600      choreService.cancelChore(compactionChecker);
2601      choreService.cancelChore(periodicFlusher);
2602      choreService.cancelChore(healthCheckChore);
2603      choreService.cancelChore(executorStatusChore);
2604      choreService.cancelChore(storefileRefresher);
2605      choreService.cancelChore(movedRegionsCleaner);
2606      choreService.cancelChore(fsUtilizationChore);
2607      choreService.cancelChore(slowLogTableOpsChore);
2608      // clean up the remaining scheduled chores (in case we missed out any)
2609      choreService.shutdown();
2610    }
2611
2612    if (this.cacheFlusher != null) {
2613      this.cacheFlusher.join();
2614    }
2615
2616    if (this.spanReceiverHost != null) {
2617      this.spanReceiverHost.closeReceivers();
2618    }
2619    if (this.walRoller != null) {
2620      this.walRoller.close();
2621    }
2622    if (this.compactSplitThread != null) {
2623      this.compactSplitThread.join();
2624    }
2625    if (this.executorService != null) this.executorService.shutdown();
2626    if (this.replicationSourceHandler != null &&
2627        this.replicationSourceHandler == this.replicationSinkHandler) {
2628      this.replicationSourceHandler.stopReplicationService();
2629    } else {
2630      if (this.replicationSourceHandler != null) {
2631        this.replicationSourceHandler.stopReplicationService();
2632      }
2633      if (this.replicationSinkHandler != null) {
2634        this.replicationSinkHandler.stopReplicationService();
2635      }
2636    }
2637  }
2638
2639  /**
2640   * @return Return the object that implements the replication
2641   * source executorService.
2642   */
2643  @Override
2644  public ReplicationSourceService getReplicationSourceService() {
2645    return replicationSourceHandler;
2646  }
2647
2648  /**
2649   * @return Return the object that implements the replication sink executorService.
2650   */
2651  public ReplicationSinkService getReplicationSinkService() {
2652    return replicationSinkHandler;
2653  }
2654
2655  /**
2656   * Get the current master from ZooKeeper and open the RPC connection to it.
2657   * To get a fresh connection, the current rssStub must be null.
2658   * Method will block until a master is available. You can break from this
2659   * block by requesting the server stop.
2660   *
2661   * @return master + port, or null if server has been stopped
2662   */
2663  private synchronized ServerName createRegionServerStatusStub() {
2664    // Create RS stub without refreshing the master node from ZK, use cached data
2665    return createRegionServerStatusStub(false);
2666  }
2667
2668  /**
2669   * Get the current master from ZooKeeper and open the RPC connection to it. To get a fresh
2670   * connection, the current rssStub must be null. Method will block until a master is available.
2671   * You can break from this block by requesting the server stop.
2672   * @param refresh If true then master address will be read from ZK, otherwise use cached data
2673   * @return master + port, or null if server has been stopped
2674   */
2675  @VisibleForTesting
2676  protected synchronized ServerName createRegionServerStatusStub(boolean refresh) {
2677    if (rssStub != null) {
2678      return masterAddressTracker.getMasterAddress();
2679    }
2680    ServerName sn = null;
2681    long previousLogTime = 0;
2682    RegionServerStatusService.BlockingInterface intRssStub = null;
2683    LockService.BlockingInterface intLockStub = null;
2684    boolean interrupted = false;
2685    try {
2686      while (keepLooping()) {
2687        sn = this.masterAddressTracker.getMasterAddress(refresh);
2688        if (sn == null) {
2689          if (!keepLooping()) {
2690            // give up with no connection.
2691            LOG.debug("No master found and cluster is stopped; bailing out");
2692            return null;
2693          }
2694          if (System.currentTimeMillis() > (previousLogTime + 1000)) {
2695            LOG.debug("No master found; retry");
2696            previousLogTime = System.currentTimeMillis();
2697          }
2698          refresh = true; // let's try pull it from ZK directly
2699          if (sleepInterrupted(200)) {
2700            interrupted = true;
2701          }
2702          continue;
2703        }
2704
2705        // If we are on the active master, use the shortcut
2706        if (this instanceof HMaster && sn.equals(getServerName())) {
2707          // Wrap the shortcut in a class providing our version to the calls where it's relevant.
2708          // Normally, RpcServer-based threadlocals do that.
2709          intRssStub = new MasterRpcServicesVersionWrapper(((HMaster)this).getMasterRpcServices());
2710          intLockStub = ((HMaster)this).getMasterRpcServices();
2711          break;
2712        }
2713        try {
2714          BlockingRpcChannel channel =
2715            this.rpcClient.createBlockingRpcChannel(sn, userProvider.getCurrent(),
2716              shortOperationTimeout);
2717          intRssStub = RegionServerStatusService.newBlockingStub(channel);
2718          intLockStub = LockService.newBlockingStub(channel);
2719          break;
2720        } catch (IOException e) {
2721          if (System.currentTimeMillis() > (previousLogTime + 1000)) {
2722            e = e instanceof RemoteException ?
2723              ((RemoteException)e).unwrapRemoteException() : e;
2724            if (e instanceof ServerNotRunningYetException) {
2725              LOG.info("Master isn't available yet, retrying");
2726            } else {
2727              LOG.warn("Unable to connect to master. Retrying. Error was:", e);
2728            }
2729            previousLogTime = System.currentTimeMillis();
2730          }
2731          if (sleepInterrupted(200)) {
2732            interrupted = true;
2733          }
2734        }
2735      }
2736    } finally {
2737      if (interrupted) {
2738        Thread.currentThread().interrupt();
2739      }
2740    }
2741    this.rssStub = intRssStub;
2742    this.lockStub = intLockStub;
2743    return sn;
2744  }
2745
2746  /**
2747   * @return True if we should break loop because cluster is going down or
2748   * this server has been stopped or hdfs has gone bad.
2749   */
2750  private boolean keepLooping() {
2751    return !this.stopped && isClusterUp();
2752  }
2753
2754  /*
2755   * Let the master know we're here Run initialization using parameters passed
2756   * us by the master.
2757   * @return A Map of key/value configurations we got from the Master else
2758   * null if we failed to register.
2759   * @throws IOException
2760   */
2761  private RegionServerStartupResponse reportForDuty() throws IOException {
2762    if (this.masterless) return RegionServerStartupResponse.getDefaultInstance();
2763    ServerName masterServerName = createRegionServerStatusStub(true);
2764    RegionServerStatusService.BlockingInterface rss = rssStub;
2765    if (masterServerName == null || rss == null) return null;
2766    RegionServerStartupResponse result = null;
2767    try {
2768      rpcServices.requestCount.reset();
2769      rpcServices.rpcGetRequestCount.reset();
2770      rpcServices.rpcScanRequestCount.reset();
2771      rpcServices.rpcMultiRequestCount.reset();
2772      rpcServices.rpcMutateRequestCount.reset();
2773      LOG.info("reportForDuty to master=" + masterServerName + " with port="
2774        + rpcServices.isa.getPort() + ", startcode=" + this.startcode);
2775      long now = EnvironmentEdgeManager.currentTime();
2776      int port = rpcServices.isa.getPort();
2777      RegionServerStartupRequest.Builder request = RegionServerStartupRequest.newBuilder();
2778      if (!StringUtils.isBlank(useThisHostnameInstead)) {
2779        request.setUseThisHostnameInstead(useThisHostnameInstead);
2780      }
2781      request.setPort(port);
2782      request.setServerStartCode(this.startcode);
2783      request.setServerCurrentTime(now);
2784      result = rss.regionServerStartup(null, request.build());
2785    } catch (ServiceException se) {
2786      IOException ioe = ProtobufUtil.getRemoteException(se);
2787      if (ioe instanceof ClockOutOfSyncException) {
2788        LOG.error(HBaseMarkers.FATAL, "Master rejected startup because clock is out of sync",
2789            ioe);
2790        // Re-throw IOE will cause RS to abort
2791        throw ioe;
2792      } else if (ioe instanceof ServerNotRunningYetException) {
2793        LOG.debug("Master is not running yet");
2794      } else {
2795        LOG.warn("error telling master we are up", se);
2796      }
2797      rssStub = null;
2798    }
2799    return result;
2800  }
2801
2802  @Override
2803  public RegionStoreSequenceIds getLastSequenceId(byte[] encodedRegionName) {
2804    try {
2805      GetLastFlushedSequenceIdRequest req =
2806          RequestConverter.buildGetLastFlushedSequenceIdRequest(encodedRegionName);
2807      RegionServerStatusService.BlockingInterface rss = rssStub;
2808      if (rss == null) { // Try to connect one more time
2809        createRegionServerStatusStub();
2810        rss = rssStub;
2811        if (rss == null) {
2812          // Still no luck, we tried
2813          LOG.warn("Unable to connect to the master to check " + "the last flushed sequence id");
2814          return RegionStoreSequenceIds.newBuilder().setLastFlushedSequenceId(HConstants.NO_SEQNUM)
2815              .build();
2816        }
2817      }
2818      GetLastFlushedSequenceIdResponse resp = rss.getLastFlushedSequenceId(null, req);
2819      return RegionStoreSequenceIds.newBuilder()
2820          .setLastFlushedSequenceId(resp.getLastFlushedSequenceId())
2821          .addAllStoreSequenceId(resp.getStoreLastFlushedSequenceIdList()).build();
2822    } catch (ServiceException e) {
2823      LOG.warn("Unable to connect to the master to check the last flushed sequence id", e);
2824      return RegionStoreSequenceIds.newBuilder().setLastFlushedSequenceId(HConstants.NO_SEQNUM)
2825          .build();
2826    }
2827  }
2828
2829  /**
2830   * Close meta region if we carry it
2831   * @param abort Whether we're running an abort.
2832   */
2833  private void closeMetaTableRegions(final boolean abort) {
2834    HRegion meta = null;
2835    this.onlineRegionsLock.writeLock().lock();
2836    try {
2837      for (Map.Entry<String, HRegion> e: onlineRegions.entrySet()) {
2838        RegionInfo hri = e.getValue().getRegionInfo();
2839        if (hri.isMetaRegion()) {
2840          meta = e.getValue();
2841        }
2842        if (meta != null) break;
2843      }
2844    } finally {
2845      this.onlineRegionsLock.writeLock().unlock();
2846    }
2847    if (meta != null) closeRegionIgnoreErrors(meta.getRegionInfo(), abort);
2848  }
2849
2850  /**
2851   * Schedule closes on all user regions.
2852   * Should be safe calling multiple times because it wont' close regions
2853   * that are already closed or that are closing.
2854   * @param abort Whether we're running an abort.
2855   */
2856  private void closeUserRegions(final boolean abort) {
2857    this.onlineRegionsLock.writeLock().lock();
2858    try {
2859      for (Map.Entry<String, HRegion> e: this.onlineRegions.entrySet()) {
2860        HRegion r = e.getValue();
2861        if (!r.getRegionInfo().isMetaRegion() && r.isAvailable()) {
2862          // Don't update zk with this close transition; pass false.
2863          closeRegionIgnoreErrors(r.getRegionInfo(), abort);
2864        }
2865      }
2866    } finally {
2867      this.onlineRegionsLock.writeLock().unlock();
2868    }
2869  }
2870
2871  /** @return the info server */
2872  public InfoServer getInfoServer() {
2873    return infoServer;
2874  }
2875
2876  /**
2877   * @return true if a stop has been requested.
2878   */
2879  @Override
2880  public boolean isStopped() {
2881    return this.stopped;
2882  }
2883
2884  @Override
2885  public boolean isStopping() {
2886    return this.stopping;
2887  }
2888
2889  @Override
2890  public Configuration getConfiguration() {
2891    return conf;
2892  }
2893
2894  protected Map<String, HRegion> getOnlineRegions() {
2895    return this.onlineRegions;
2896  }
2897
2898  public int getNumberOfOnlineRegions() {
2899    return this.onlineRegions.size();
2900  }
2901
2902  /**
2903   * For tests, web ui and metrics.
2904   * This method will only work if HRegionServer is in the same JVM as client;
2905   * HRegion cannot be serialized to cross an rpc.
2906   */
2907  public Collection<HRegion> getOnlineRegionsLocalContext() {
2908    Collection<HRegion> regions = this.onlineRegions.values();
2909    return Collections.unmodifiableCollection(regions);
2910  }
2911
2912  @Override
2913  public void addRegion(HRegion region) {
2914    this.onlineRegions.put(region.getRegionInfo().getEncodedName(), region);
2915    configurationManager.registerObserver(region);
2916  }
2917
2918  private void addRegion(SortedMap<Long, Collection<HRegion>> sortedRegions, HRegion region,
2919      long size) {
2920    if (!sortedRegions.containsKey(size)) {
2921      sortedRegions.put(size, new ArrayList<>());
2922    }
2923    sortedRegions.get(size).add(region);
2924  }
2925  /**
2926   * @return A new Map of online regions sorted by region off-heap size with the first entry being
2927   *   the biggest.
2928   */
2929  SortedMap<Long, Collection<HRegion>> getCopyOfOnlineRegionsSortedByOffHeapSize() {
2930    // we'll sort the regions in reverse
2931    SortedMap<Long, Collection<HRegion>> sortedRegions = new TreeMap<>(Comparator.reverseOrder());
2932    // Copy over all regions. Regions are sorted by size with biggest first.
2933    for (HRegion region : this.onlineRegions.values()) {
2934      addRegion(sortedRegions, region, region.getMemStoreOffHeapSize());
2935    }
2936    return sortedRegions;
2937  }
2938
2939  /**
2940   * @return A new Map of online regions sorted by region heap size with the first entry being the
2941   *   biggest.
2942   */
2943  SortedMap<Long, Collection<HRegion>> getCopyOfOnlineRegionsSortedByOnHeapSize() {
2944    // we'll sort the regions in reverse
2945    SortedMap<Long, Collection<HRegion>> sortedRegions = new TreeMap<>(Comparator.reverseOrder());
2946    // Copy over all regions. Regions are sorted by size with biggest first.
2947    for (HRegion region : this.onlineRegions.values()) {
2948      addRegion(sortedRegions, region, region.getMemStoreHeapSize());
2949    }
2950    return sortedRegions;
2951  }
2952
2953  /**
2954   * @return time stamp in millis of when this region server was started
2955   */
2956  public long getStartcode() {
2957    return this.startcode;
2958  }
2959
2960  /** @return reference to FlushRequester */
2961  @Override
2962  public FlushRequester getFlushRequester() {
2963    return this.cacheFlusher;
2964  }
2965
2966  @Override
2967  public CompactionRequester getCompactionRequestor() {
2968    return this.compactSplitThread;
2969  }
2970
2971  @Override
2972  public LeaseManager getLeaseManager() {
2973    return leaseManager;
2974  }
2975
2976  /**
2977   * @return Return the rootDir.
2978   */
2979  protected Path getDataRootDir() {
2980    return dataRootDir;
2981  }
2982
2983  @Override
2984  public FileSystem getFileSystem() {
2985    return dataFs;
2986  }
2987
2988  /**
2989   * @return {@code true} when the data file system is available, {@code false} otherwise.
2990   */
2991  boolean isDataFileSystemOk() {
2992    return this.dataFsOk;
2993  }
2994
2995  /**
2996   * @return Return the walRootDir.
2997   */
2998  public Path getWALRootDir() {
2999    return walRootDir;
3000  }
3001
3002  /**
3003   * @return Return the walFs.
3004   */
3005  public FileSystem getWALFileSystem() {
3006    return walFs;
3007  }
3008
3009  @Override
3010  public String toString() {
3011    return getServerName().toString();
3012  }
3013
3014  @Override
3015  public ZKWatcher getZooKeeper() {
3016    return zooKeeper;
3017  }
3018
3019  @Override
3020  public CoordinatedStateManager getCoordinatedStateManager() {
3021    return csm;
3022  }
3023
3024  @Override
3025  public ServerName getServerName() {
3026    return serverName;
3027  }
3028
3029  public RegionServerCoprocessorHost getRegionServerCoprocessorHost(){
3030    return this.rsHost;
3031  }
3032
3033  @Override
3034  public ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS() {
3035    return this.regionsInTransitionInRS;
3036  }
3037
3038  @Override
3039  public ExecutorService getExecutorService() {
3040    return executorService;
3041  }
3042
3043  @Override
3044  public ChoreService getChoreService() {
3045    return choreService;
3046  }
3047
3048  @Override
3049  public RegionServerRpcQuotaManager getRegionServerRpcQuotaManager() {
3050    return rsQuotaManager;
3051  }
3052
3053  //
3054  // Main program and support routines
3055  //
3056  /**
3057   * Load the replication executorService objects, if any
3058   */
3059  private static void createNewReplicationInstance(Configuration conf, HRegionServer server,
3060      FileSystem walFs, Path walDir, Path oldWALDir, WALProvider walProvider) throws IOException {
3061    // read in the name of the source replication class from the config file.
3062    String sourceClassname = conf.get(HConstants.REPLICATION_SOURCE_SERVICE_CLASSNAME,
3063      HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
3064
3065    // read in the name of the sink replication class from the config file.
3066    String sinkClassname = conf.get(HConstants.REPLICATION_SINK_SERVICE_CLASSNAME,
3067      HConstants.REPLICATION_SERVICE_CLASSNAME_DEFAULT);
3068
3069    // If both the sink and the source class names are the same, then instantiate
3070    // only one object.
3071    if (sourceClassname.equals(sinkClassname)) {
3072      server.replicationSourceHandler = newReplicationInstance(sourceClassname,
3073        ReplicationSourceService.class, conf, server, walFs, walDir, oldWALDir, walProvider);
3074      server.replicationSinkHandler = (ReplicationSinkService) server.replicationSourceHandler;
3075    } else {
3076      server.replicationSourceHandler = newReplicationInstance(sourceClassname,
3077        ReplicationSourceService.class, conf, server, walFs, walDir, oldWALDir, walProvider);
3078      server.replicationSinkHandler = newReplicationInstance(sinkClassname,
3079        ReplicationSinkService.class, conf, server, walFs, walDir, oldWALDir, walProvider);
3080    }
3081  }
3082
3083  private static <T extends ReplicationService> T newReplicationInstance(String classname,
3084      Class<T> xface, Configuration conf, HRegionServer server, FileSystem walFs, Path logDir,
3085      Path oldLogDir, WALProvider walProvider) throws IOException {
3086    final Class<? extends T> clazz;
3087    try {
3088      ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
3089      clazz = Class.forName(classname, true, classLoader).asSubclass(xface);
3090    } catch (java.lang.ClassNotFoundException nfe) {
3091      throw new IOException("Could not find class for " + classname);
3092    }
3093    T service = ReflectionUtils.newInstance(clazz, conf);
3094    service.initialize(server, walFs, logDir, oldLogDir, walProvider);
3095    return service;
3096  }
3097
3098  public Map<String, ReplicationStatus> getWalGroupsReplicationStatus(){
3099    Map<String, ReplicationStatus> walGroupsReplicationStatus = new TreeMap<>();
3100    if(!this.isOnline()){
3101      return walGroupsReplicationStatus;
3102    }
3103    List<ReplicationSourceInterface> allSources = new ArrayList<>();
3104    allSources.addAll(replicationSourceHandler.getReplicationManager().getSources());
3105    allSources.addAll(replicationSourceHandler.getReplicationManager().getOldSources());
3106    for(ReplicationSourceInterface source: allSources){
3107      walGroupsReplicationStatus.putAll(source.getWalGroupStatus());
3108    }
3109    return walGroupsReplicationStatus;
3110  }
3111
3112  /**
3113   * Utility for constructing an instance of the passed HRegionServer class.
3114   */
3115  static HRegionServer constructRegionServer(
3116      final Class<? extends HRegionServer> regionServerClass,
3117      final Configuration conf
3118  ) {
3119    try {
3120      Constructor<? extends HRegionServer> c =
3121          regionServerClass.getConstructor(Configuration.class);
3122      return c.newInstance(conf);
3123    } catch (Exception e) {
3124      throw new RuntimeException("Failed construction of " + "Regionserver: "
3125          + regionServerClass.toString(), e);
3126    }
3127  }
3128
3129  /**
3130   * @see org.apache.hadoop.hbase.regionserver.HRegionServerCommandLine
3131   */
3132  public static void main(String[] args) {
3133    LOG.info("STARTING executorService " + HRegionServer.class.getSimpleName());
3134    VersionInfo.logVersion();
3135    Configuration conf = HBaseConfiguration.create();
3136    @SuppressWarnings("unchecked")
3137    Class<? extends HRegionServer> regionServerClass = (Class<? extends HRegionServer>) conf
3138        .getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class);
3139
3140    new HRegionServerCommandLine(regionServerClass).doMain(args);
3141  }
3142
3143  /**
3144   * Gets the online regions of the specified table.
3145   * This method looks at the in-memory onlineRegions.  It does not go to <code>hbase:meta</code>.
3146   * Only returns <em>online</em> regions.  If a region on this table has been
3147   * closed during a disable, etc., it will not be included in the returned list.
3148   * So, the returned list may not necessarily be ALL regions in this table, its
3149   * all the ONLINE regions in the table.
3150   * @param tableName table to limit the scope of the query
3151   * @return Online regions from <code>tableName</code>
3152   */
3153  @Override
3154  public List<HRegion> getRegions(TableName tableName) {
3155     List<HRegion> tableRegions = new ArrayList<>();
3156     synchronized (this.onlineRegions) {
3157       for (HRegion region: this.onlineRegions.values()) {
3158         RegionInfo regionInfo = region.getRegionInfo();
3159         if(regionInfo.getTable().equals(tableName)) {
3160           tableRegions.add(region);
3161         }
3162       }
3163     }
3164     return tableRegions;
3165   }
3166
3167  @Override
3168  public List<HRegion> getRegions() {
3169    List<HRegion> allRegions;
3170    synchronized (this.onlineRegions) {
3171      // Return a clone copy of the onlineRegions
3172      allRegions = new ArrayList<>(onlineRegions.values());
3173    }
3174    return allRegions;
3175  }
3176
3177  /**
3178   * Gets the online tables in this RS.
3179   * This method looks at the in-memory onlineRegions.
3180   * @return all the online tables in this RS
3181   */
3182  public Set<TableName> getOnlineTables() {
3183    Set<TableName> tables = new HashSet<>();
3184    synchronized (this.onlineRegions) {
3185      for (Region region: this.onlineRegions.values()) {
3186        tables.add(region.getTableDescriptor().getTableName());
3187      }
3188    }
3189    return tables;
3190  }
3191
3192  public String[] getRegionServerCoprocessors() {
3193    TreeSet<String> coprocessors = new TreeSet<>();
3194    try {
3195      coprocessors.addAll(getWAL(null).getCoprocessorHost().getCoprocessors());
3196    } catch (IOException exception) {
3197      LOG.warn("Exception attempting to fetch wal coprocessor information for the common wal; " +
3198          "skipping.");
3199      LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception);
3200    }
3201    Collection<HRegion> regions = getOnlineRegionsLocalContext();
3202    for (HRegion region: regions) {
3203      coprocessors.addAll(region.getCoprocessorHost().getCoprocessors());
3204      try {
3205        coprocessors.addAll(getWAL(region.getRegionInfo()).getCoprocessorHost().getCoprocessors());
3206      } catch (IOException exception) {
3207        LOG.warn("Exception attempting to fetch wal coprocessor information for region " + region +
3208            "; skipping.");
3209        LOG.debug("Exception details for failure to fetch wal coprocessor information.", exception);
3210      }
3211    }
3212    coprocessors.addAll(rsHost.getCoprocessors());
3213    return coprocessors.toArray(new String[0]);
3214  }
3215
3216  /**
3217   * Try to close the region, logs a warning on failure but continues.
3218   * @param region Region to close
3219   */
3220  private void closeRegionIgnoreErrors(RegionInfo region, final boolean abort) {
3221    try {
3222      if (!closeRegion(region.getEncodedName(), abort, null)) {
3223        LOG.warn("Failed to close " + region.getRegionNameAsString() +
3224            " - ignoring and continuing");
3225      }
3226    } catch (IOException e) {
3227      LOG.warn("Failed to close " + region.getRegionNameAsString() +
3228          " - ignoring and continuing", e);
3229    }
3230  }
3231
3232  /**
3233   * Close asynchronously a region, can be called from the master or internally by the regionserver
3234   * when stopping. If called from the master, the region will update the status.
3235   *
3236   * <p>
3237   * If an opening was in progress, this method will cancel it, but will not start a new close. The
3238   * coprocessors are not called in this case. A NotServingRegionException exception is thrown.
3239   * </p>
3240
3241   * <p>
3242   *   If a close was in progress, this new request will be ignored, and an exception thrown.
3243   * </p>
3244   *
3245   * @param encodedName Region to close
3246   * @param abort True if we are aborting
3247   * @param destination Where the Region is being moved too... maybe null if unknown.
3248   * @return True if closed a region.
3249   * @throws NotServingRegionException if the region is not online
3250   */
3251  protected boolean closeRegion(String encodedName, final boolean abort,
3252        final ServerName destination)
3253      throws NotServingRegionException {
3254    //Check for permissions to close.
3255    HRegion actualRegion = this.getRegion(encodedName);
3256    // Can be null if we're calling close on a region that's not online
3257    if ((actualRegion != null) && (actualRegion.getCoprocessorHost() != null)) {
3258      try {
3259        actualRegion.getCoprocessorHost().preClose(false);
3260      } catch (IOException exp) {
3261        LOG.warn("Unable to close region: the coprocessor launched an error ", exp);
3262        return false;
3263      }
3264    }
3265
3266    // previous can come back 'null' if not in map.
3267    final Boolean previous = this.regionsInTransitionInRS.putIfAbsent(Bytes.toBytes(encodedName),
3268        Boolean.FALSE);
3269
3270    if (Boolean.TRUE.equals(previous)) {
3271      LOG.info("Received CLOSE for the region:" + encodedName + " , which we are already " +
3272          "trying to OPEN. Cancelling OPENING.");
3273      if (!regionsInTransitionInRS.replace(Bytes.toBytes(encodedName), previous, Boolean.FALSE)) {
3274        // The replace failed. That should be an exceptional case, but theoretically it can happen.
3275        // We're going to try to do a standard close then.
3276        LOG.warn("The opening for region " + encodedName + " was done before we could cancel it." +
3277            " Doing a standard close now");
3278        return closeRegion(encodedName, abort, destination);
3279      }
3280      // Let's get the region from the online region list again
3281      actualRegion = this.getRegion(encodedName);
3282      if (actualRegion == null) { // If already online, we still need to close it.
3283        LOG.info("The opening previously in progress has been cancelled by a CLOSE request.");
3284        // The master deletes the znode when it receives this exception.
3285        throw new NotServingRegionException("The region " + encodedName +
3286          " was opening but not yet served. Opening is cancelled.");
3287      }
3288    } else if (previous == null) {
3289      LOG.info("Received CLOSE for {}", encodedName);
3290    } else if (Boolean.FALSE.equals(previous)) {
3291      LOG.info("Received CLOSE for the region: " + encodedName +
3292        ", which we are already trying to CLOSE, but not completed yet");
3293      return true;
3294    }
3295
3296    if (actualRegion == null) {
3297      LOG.debug("Received CLOSE for a region which is not online, and we're not opening.");
3298      this.regionsInTransitionInRS.remove(Bytes.toBytes(encodedName));
3299      // The master deletes the znode when it receives this exception.
3300      throw new NotServingRegionException("The region " + encodedName +
3301          " is not online, and is not opening.");
3302    }
3303
3304    CloseRegionHandler crh;
3305    final RegionInfo hri = actualRegion.getRegionInfo();
3306    if (hri.isMetaRegion()) {
3307      crh = new CloseMetaHandler(this, this, hri, abort);
3308    } else {
3309      crh = new CloseRegionHandler(this, this, hri, abort, destination);
3310    }
3311    this.executorService.submit(crh);
3312    return true;
3313  }
3314
3315   /**
3316   * @return HRegion for the passed binary <code>regionName</code> or null if
3317   *         named region is not member of the online regions.
3318   */
3319  public HRegion getOnlineRegion(final byte[] regionName) {
3320    String encodedRegionName = RegionInfo.encodeRegionName(regionName);
3321    return this.onlineRegions.get(encodedRegionName);
3322  }
3323
3324  @Override
3325  public HRegion getRegion(final String encodedRegionName) {
3326    return this.onlineRegions.get(encodedRegionName);
3327  }
3328
3329
3330  @Override
3331  public boolean removeRegion(final HRegion r, ServerName destination) {
3332    HRegion toReturn = this.onlineRegions.remove(r.getRegionInfo().getEncodedName());
3333    metricsRegionServerImpl.requestsCountCache.remove(r.getRegionInfo().getEncodedName());
3334    if (destination != null) {
3335      long closeSeqNum = r.getMaxFlushedSeqId();
3336      if (closeSeqNum == HConstants.NO_SEQNUM) {
3337        // No edits in WAL for this region; get the sequence number when the region was opened.
3338        closeSeqNum = r.getOpenSeqNum();
3339        if (closeSeqNum == HConstants.NO_SEQNUM) closeSeqNum = 0;
3340      }
3341      boolean selfMove = ServerName.isSameAddress(destination, this.getServerName());
3342      addToMovedRegions(r.getRegionInfo().getEncodedName(), destination, closeSeqNum, selfMove);
3343      if (selfMove) {
3344        this.regionServerAccounting.getRetainedRegionRWRequestsCnt().put(r.getRegionInfo().getEncodedName()
3345          , new Pair<>(r.getReadRequestsCount(), r.getWriteRequestsCount()));
3346      }
3347    }
3348    this.regionFavoredNodesMap.remove(r.getRegionInfo().getEncodedName());
3349    configurationManager.deregisterObserver(r);
3350    return toReturn != null;
3351  }
3352
3353  /**
3354   * Protected Utility method for safely obtaining an HRegion handle.
3355   *
3356   * @param regionName Name of online {@link HRegion} to return
3357   * @return {@link HRegion} for <code>regionName</code>
3358   */
3359  protected HRegion getRegion(final byte[] regionName)
3360      throws NotServingRegionException {
3361    String encodedRegionName = RegionInfo.encodeRegionName(regionName);
3362    return getRegionByEncodedName(regionName, encodedRegionName);
3363  }
3364
3365  public HRegion getRegionByEncodedName(String encodedRegionName)
3366      throws NotServingRegionException {
3367    return getRegionByEncodedName(null, encodedRegionName);
3368  }
3369
3370  private HRegion getRegionByEncodedName(byte[] regionName, String encodedRegionName)
3371    throws NotServingRegionException {
3372    HRegion region = this.onlineRegions.get(encodedRegionName);
3373    if (region == null) {
3374      MovedRegionInfo moveInfo = getMovedRegion(encodedRegionName);
3375      if (moveInfo != null) {
3376        throw new RegionMovedException(moveInfo.getServerName(), moveInfo.getSeqNum());
3377      }
3378      Boolean isOpening = this.regionsInTransitionInRS.get(Bytes.toBytes(encodedRegionName));
3379      String regionNameStr = regionName == null?
3380        encodedRegionName: Bytes.toStringBinary(regionName);
3381      if (isOpening != null && isOpening) {
3382        throw new RegionOpeningException("Region " + regionNameStr +
3383          " is opening on " + this.serverName);
3384      }
3385      throw new NotServingRegionException("" + regionNameStr +
3386        " is not online on " + this.serverName);
3387    }
3388    return region;
3389  }
3390
3391  /**
3392   * Cleanup after Throwable caught invoking method. Converts <code>t</code> to
3393   * IOE if it isn't already.
3394   *
3395   * @param t Throwable
3396   * @param msg Message to log in error. Can be null.
3397   * @return Throwable converted to an IOE; methods can only let out IOEs.
3398   */
3399  private Throwable cleanup(final Throwable t, final String msg) {
3400    // Don't log as error if NSRE; NSRE is 'normal' operation.
3401    if (t instanceof NotServingRegionException) {
3402      LOG.debug("NotServingRegionException; " + t.getMessage());
3403      return t;
3404    }
3405    Throwable e = t instanceof RemoteException ? ((RemoteException) t).unwrapRemoteException() : t;
3406    if (msg == null) {
3407      LOG.error("", e);
3408    } else {
3409      LOG.error(msg, e);
3410    }
3411    if (!rpcServices.checkOOME(t)) {
3412      checkFileSystem();
3413    }
3414    return t;
3415  }
3416
3417  /**
3418   * @param msg Message to put in new IOE if passed <code>t</code> is not an IOE
3419   * @return Make <code>t</code> an IOE if it isn't already.
3420   */
3421  private IOException convertThrowableToIOE(final Throwable t, final String msg) {
3422    return (t instanceof IOException ? (IOException) t : msg == null
3423        || msg.length() == 0 ? new IOException(t) : new IOException(msg, t));
3424  }
3425
3426  /**
3427   * Checks to see if the file system is still accessible. If not, sets
3428   * abortRequested and stopRequested
3429   *
3430   * @return false if file system is not available
3431   */
3432  boolean checkFileSystem() {
3433    if (this.dataFsOk && this.dataFs != null) {
3434      try {
3435        FSUtils.checkFileSystemAvailable(this.dataFs);
3436      } catch (IOException e) {
3437        abort("File System not available", e);
3438        this.dataFsOk = false;
3439      }
3440    }
3441    return this.dataFsOk;
3442  }
3443
3444  @Override
3445  public void updateRegionFavoredNodesMapping(String encodedRegionName,
3446      List<org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.ServerName> favoredNodes) {
3447    InetSocketAddress[] addr = new InetSocketAddress[favoredNodes.size()];
3448    // Refer to the comment on the declaration of regionFavoredNodesMap on why
3449    // it is a map of region name to InetSocketAddress[]
3450    for (int i = 0; i < favoredNodes.size(); i++) {
3451      addr[i] = InetSocketAddress.createUnresolved(favoredNodes.get(i).getHostName(),
3452          favoredNodes.get(i).getPort());
3453    }
3454    regionFavoredNodesMap.put(encodedRegionName, addr);
3455  }
3456
3457  /**
3458   * Return the favored nodes for a region given its encoded name. Look at the
3459   * comment around {@link #regionFavoredNodesMap} on why it is InetSocketAddress[]
3460   *
3461   * @return array of favored locations
3462   */
3463  @Override
3464  public InetSocketAddress[] getFavoredNodesForRegion(String encodedRegionName) {
3465    return regionFavoredNodesMap.get(encodedRegionName);
3466  }
3467
3468  @Override
3469  public ServerNonceManager getNonceManager() {
3470    return this.nonceManager;
3471  }
3472
3473  private static class MovedRegionInfo {
3474    private final ServerName serverName;
3475    private final long seqNum;
3476    private final long moveTime;
3477
3478    MovedRegionInfo(ServerName serverName, long closeSeqNum) {
3479      this.serverName = serverName;
3480      this.seqNum = closeSeqNum;
3481      this.moveTime = EnvironmentEdgeManager.currentTime();
3482     }
3483
3484    public ServerName getServerName() {
3485      return serverName;
3486    }
3487
3488    public long getSeqNum() {
3489      return seqNum;
3490    }
3491
3492    long getMoveTime() {
3493      return moveTime;
3494    }
3495  }
3496
3497  /**
3498   * This map will contains all the regions that we closed for a move.
3499   * We add the time it was moved as we don't want to keep too old information
3500   */
3501  private Map<String, MovedRegionInfo> movedRegions = new ConcurrentHashMap<>(3000);
3502
3503  /**
3504   * We need a timeout. If not there is a risk of giving a wrong information: this would double
3505   * the number of network calls instead of reducing them.
3506   */
3507  private static final int TIMEOUT_REGION_MOVED = (2 * 60 * 1000);
3508
3509  private void addToMovedRegions(String encodedName, ServerName destination, long closeSeqNum, boolean selfMove) {
3510    if (selfMove) {
3511      LOG.warn("Not adding moved region record: " + encodedName + " to self.");
3512      return;
3513    }
3514    LOG.info("Adding " + encodedName + " move to " + destination + " record at close sequenceid=" +
3515        closeSeqNum);
3516    movedRegions.put(encodedName, new MovedRegionInfo(destination, closeSeqNum));
3517  }
3518
3519  void removeFromMovedRegions(String encodedName) {
3520    movedRegions.remove(encodedName);
3521  }
3522
3523  private MovedRegionInfo getMovedRegion(final String encodedRegionName) {
3524    MovedRegionInfo dest = movedRegions.get(encodedRegionName);
3525
3526    long now = EnvironmentEdgeManager.currentTime();
3527    if (dest != null) {
3528      if (dest.getMoveTime() > (now - TIMEOUT_REGION_MOVED)) {
3529        return dest;
3530      } else {
3531        movedRegions.remove(encodedRegionName);
3532      }
3533    }
3534
3535    return null;
3536  }
3537
3538  /**
3539   * Remove the expired entries from the moved regions list.
3540   */
3541  protected void cleanMovedRegions() {
3542    final long cutOff = System.currentTimeMillis() - TIMEOUT_REGION_MOVED;
3543
3544    movedRegions.entrySet().removeIf(e -> e.getValue().getMoveTime() < cutOff);
3545  }
3546
3547  /*
3548   * Use this to allow tests to override and schedule more frequently.
3549   */
3550
3551  protected int movedRegionCleanerPeriod() {
3552        return TIMEOUT_REGION_MOVED;
3553  }
3554
3555  /**
3556   * Creates a Chore thread to clean the moved region cache.
3557   */
3558  protected final static class MovedRegionsCleaner extends ScheduledChore implements Stoppable {
3559    private HRegionServer regionServer;
3560    Stoppable stoppable;
3561
3562    private MovedRegionsCleaner(
3563      HRegionServer regionServer, Stoppable stoppable){
3564      super("MovedRegionsCleaner for region " + regionServer, stoppable,
3565          regionServer.movedRegionCleanerPeriod());
3566      this.regionServer = regionServer;
3567      this.stoppable = stoppable;
3568    }
3569
3570    static MovedRegionsCleaner create(HRegionServer rs){
3571      Stoppable stoppable = new Stoppable() {
3572        private volatile boolean isStopped = false;
3573        @Override public void stop(String why) { isStopped = true;}
3574        @Override public boolean isStopped() {return isStopped;}
3575      };
3576
3577      return new MovedRegionsCleaner(rs, stoppable);
3578    }
3579
3580    @Override
3581    protected void chore() {
3582      regionServer.cleanMovedRegions();
3583    }
3584
3585    @Override
3586    public void stop(String why) {
3587      stoppable.stop(why);
3588    }
3589
3590    @Override
3591    public boolean isStopped() {
3592      return stoppable.isStopped();
3593    }
3594  }
3595
3596  private String getMyEphemeralNodePath() {
3597    return ZNodePaths.joinZNode(this.zooKeeper.getZNodePaths().rsZNode, getServerName().toString());
3598  }
3599
3600  private boolean isHealthCheckerConfigured() {
3601    String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
3602    return org.apache.commons.lang3.StringUtils.isNotBlank(healthScriptLocation);
3603  }
3604
3605  /**
3606   * @return the underlying {@link CompactSplit} for the servers
3607   */
3608  public CompactSplit getCompactSplitThread() {
3609    return this.compactSplitThread;
3610  }
3611
3612  CoprocessorServiceResponse execRegionServerService(
3613      @SuppressWarnings("UnusedParameters") final RpcController controller,
3614      final CoprocessorServiceRequest serviceRequest) throws ServiceException {
3615    try {
3616      ServerRpcController serviceController = new ServerRpcController();
3617      CoprocessorServiceCall call = serviceRequest.getCall();
3618      String serviceName = call.getServiceName();
3619      Service service = coprocessorServiceHandlers.get(serviceName);
3620      if (service == null) {
3621        throw new UnknownProtocolException(null, "No registered coprocessor executorService found for " +
3622            serviceName);
3623      }
3624      ServiceDescriptor serviceDesc =
3625          service.getDescriptorForType();
3626
3627      String methodName = call.getMethodName();
3628      MethodDescriptor methodDesc =
3629          serviceDesc.findMethodByName(methodName);
3630      if (methodDesc == null) {
3631        throw new UnknownProtocolException(service.getClass(), "Unknown method " + methodName +
3632            " called on executorService " + serviceName);
3633      }
3634
3635      Message request =
3636          CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
3637      final Message.Builder responseBuilder =
3638          service.getResponsePrototype(methodDesc).newBuilderForType();
3639      service.callMethod(methodDesc, serviceController, request, message -> {
3640        if (message != null) {
3641          responseBuilder.mergeFrom(message);
3642        }
3643      });
3644      IOException exception = CoprocessorRpcUtils.getControllerException(serviceController);
3645      if (exception != null) {
3646        throw exception;
3647      }
3648      return CoprocessorRpcUtils.getResponse(responseBuilder.build(), HConstants.EMPTY_BYTE_ARRAY);
3649    } catch (IOException ie) {
3650      throw new ServiceException(ie);
3651    }
3652  }
3653
3654  /**
3655   * May be null if this is a master which not carry table.
3656   *
3657   * @return The block cache instance used by the regionserver.
3658   */
3659  @Override
3660  public Optional<BlockCache> getBlockCache() {
3661    return Optional.ofNullable(this.blockCache);
3662  }
3663
3664  /**
3665   * May be null if this is a master which not carry table.
3666   *
3667   * @return The cache for mob files used by the regionserver.
3668   */
3669  @Override
3670  public Optional<MobFileCache> getMobFileCache() {
3671    return Optional.ofNullable(this.mobFileCache);
3672  }
3673
3674  @Override
3675  public AccessChecker getAccessChecker() {
3676    return rpcServices.getAccessChecker();
3677  }
3678
3679  @Override
3680  public ZKPermissionWatcher getZKPermissionWatcher() {
3681    return rpcServices.getZkPermissionWatcher();
3682  }
3683
3684  /**
3685   * @return : Returns the ConfigurationManager object for testing purposes.
3686   */
3687  @VisibleForTesting
3688  ConfigurationManager getConfigurationManager() {
3689    return configurationManager;
3690  }
3691
3692  /**
3693   * @return Return table descriptors implementation.
3694   */
3695  @Override
3696  public TableDescriptors getTableDescriptors() {
3697    return this.tableDescriptors;
3698  }
3699
3700  /**
3701   * Reload the configuration from disk.
3702   */
3703  void updateConfiguration() {
3704    LOG.info("Reloading the configuration from disk.");
3705    // Reload the configuration from disk.
3706    conf.reloadConfiguration();
3707    configurationManager.notifyAllObservers(conf);
3708  }
3709
3710  CacheEvictionStats clearRegionBlockCache(Region region) {
3711    long evictedBlocks = 0;
3712
3713    for(Store store : region.getStores()) {
3714      for(StoreFile hFile : store.getStorefiles()) {
3715        evictedBlocks += blockCache.evictBlocksByHfileName(hFile.getPath().getName());
3716      }
3717    }
3718
3719    return CacheEvictionStats.builder()
3720        .withEvictedBlocks(evictedBlocks)
3721        .build();
3722  }
3723
3724  @Override
3725  public double getCompactionPressure() {
3726    double max = 0;
3727    for (Region region : onlineRegions.values()) {
3728      for (Store store : region.getStores()) {
3729        double normCount = store.getCompactionPressure();
3730        if (normCount > max) {
3731          max = normCount;
3732        }
3733      }
3734    }
3735    return max;
3736  }
3737
3738  @Override
3739  public HeapMemoryManager getHeapMemoryManager() {
3740    return hMemManager;
3741  }
3742
3743  MemStoreFlusher getMemStoreFlusher() {
3744    return cacheFlusher;
3745  }
3746
3747  /**
3748   * For testing
3749   * @return whether all wal roll request finished for this regionserver
3750   */
3751  @VisibleForTesting
3752  public boolean walRollRequestFinished() {
3753    return this.walRoller.walRollFinished();
3754  }
3755
3756  @Override
3757  public ThroughputController getFlushThroughputController() {
3758    return flushThroughputController;
3759  }
3760
3761  @Override
3762  public double getFlushPressure() {
3763    if (getRegionServerAccounting() == null || cacheFlusher == null) {
3764      // return 0 during RS initialization
3765      return 0.0;
3766    }
3767    return getRegionServerAccounting().getFlushPressure();
3768  }
3769
3770  @Override
3771  public void onConfigurationChange(Configuration newConf) {
3772    ThroughputController old = this.flushThroughputController;
3773    if (old != null) {
3774      old.stop("configuration change");
3775    }
3776    this.flushThroughputController = FlushThroughputControllerFactory.create(this, newConf);
3777    try {
3778      Superusers.initialize(newConf);
3779    } catch (IOException e) {
3780      LOG.warn("Failed to initialize SuperUsers on reloading of the configuration");
3781    }
3782  }
3783
3784  @Override
3785  public MetricsRegionServer getMetrics() {
3786    return metricsRegionServer;
3787  }
3788
3789  @Override
3790  public SecureBulkLoadManager getSecureBulkLoadManager() {
3791    return this.secureBulkLoadManager;
3792  }
3793
3794  @Override
3795  public EntityLock regionLock(final List<RegionInfo> regionInfo, final String description,
3796      final Abortable abort) {
3797    final LockServiceClient client =
3798        new LockServiceClient(conf, lockStub, asyncClusterConnection.getNonceGenerator());
3799    return client.regionLock(regionInfo, description, abort);
3800  }
3801
3802  @Override
3803  public void unassign(byte[] regionName) throws IOException {
3804    FutureUtils.get(asyncClusterConnection.getAdmin().unassign(regionName, false));
3805  }
3806
3807  @Override
3808  public RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager() {
3809    return this.rsSpaceQuotaManager;
3810  }
3811
3812  @Override
3813  public boolean reportFileArchivalForQuotas(TableName tableName,
3814      Collection<Entry<String, Long>> archivedFiles) {
3815    RegionServerStatusService.BlockingInterface rss = rssStub;
3816    if (rss == null || rsSpaceQuotaManager == null) {
3817      // the current server could be stopping.
3818      LOG.trace("Skipping file archival reporting to HMaster as stub is null");
3819      return false;
3820    }
3821    try {
3822      RegionServerStatusProtos.FileArchiveNotificationRequest request =
3823          rsSpaceQuotaManager.buildFileArchiveRequest(tableName, archivedFiles);
3824      rss.reportFileArchival(null, request);
3825    } catch (ServiceException se) {
3826      IOException ioe = ProtobufUtil.getRemoteException(se);
3827      if (ioe instanceof PleaseHoldException) {
3828        if (LOG.isTraceEnabled()) {
3829          LOG.trace("Failed to report file archival(s) to Master because it is initializing."
3830              + " This will be retried.", ioe);
3831        }
3832        // The Master is coming up. Will retry the report later. Avoid re-creating the stub.
3833        return false;
3834      }
3835      if (rssStub == rss) {
3836        rssStub = null;
3837      }
3838      // re-create the stub if we failed to report the archival
3839      createRegionServerStatusStub(true);
3840      LOG.debug("Failed to report file archival(s) to Master. This will be retried.", ioe);
3841      return false;
3842    }
3843    return true;
3844  }
3845
3846  public NettyEventLoopGroupConfig getEventLoopGroupConfig() {
3847    return eventLoopGroupConfig;
3848  }
3849
3850  @Override
3851  public Connection createConnection(Configuration conf) throws IOException {
3852    User user = UserProvider.instantiate(conf).getCurrent();
3853    return ConnectionFactory.createConnection(conf, null, user);
3854  }
3855
3856  void executeProcedure(long procId, RSProcedureCallable callable) {
3857    executorService.submit(new RSProcedureHandler(this, procId, callable));
3858  }
3859
3860  public void remoteProcedureComplete(long procId, Throwable error) {
3861    procedureResultReporter.complete(procId, error);
3862  }
3863
3864  void reportProcedureDone(ReportProcedureDoneRequest request) throws IOException {
3865    RegionServerStatusService.BlockingInterface rss;
3866    // TODO: juggling class state with an instance variable, outside of a synchronized block :'(
3867    for (;;) {
3868      rss = rssStub;
3869      if (rss != null) {
3870        break;
3871      }
3872      createRegionServerStatusStub();
3873    }
3874    try {
3875      rss.reportProcedureDone(null, request);
3876    } catch (ServiceException se) {
3877      if (rssStub == rss) {
3878        rssStub = null;
3879      }
3880      throw ProtobufUtil.getRemoteException(se);
3881    }
3882  }
3883
3884  /**
3885   * Will ignore the open/close region procedures which already submitted or executed.
3886   *
3887   * When master had unfinished open/close region procedure and restarted, new active master may
3888   * send duplicate open/close region request to regionserver. The open/close request is submitted
3889   * to a thread pool and execute. So first need a cache for submitted open/close region procedures.
3890   *
3891   * After the open/close region request executed and report region transition succeed, cache it in
3892   * executed region procedures cache. See {@link #finishRegionProcedure(long)}. After report region
3893   * transition succeed, master will not send the open/close region request to regionserver again.
3894   * And we thought that the ongoing duplicate open/close region request should not be delayed more
3895   * than 600 seconds. So the executed region procedures cache will expire after 600 seconds.
3896   *
3897   * See HBASE-22404 for more details.
3898   *
3899   * @param procId the id of the open/close region procedure
3900   * @return true if the procedure can be submitted.
3901   */
3902  boolean submitRegionProcedure(long procId) {
3903    if (procId == -1) {
3904      return true;
3905    }
3906    // Ignore the region procedures which already submitted.
3907    Long previous = submittedRegionProcedures.putIfAbsent(procId, procId);
3908    if (previous != null) {
3909      LOG.warn("Received procedure pid={}, which already submitted, just ignore it", procId);
3910      return false;
3911    }
3912    // Ignore the region procedures which already executed.
3913    if (executedRegionProcedures.getIfPresent(procId) != null) {
3914      LOG.warn("Received procedure pid={}, which already executed, just ignore it", procId);
3915      return false;
3916    }
3917    return true;
3918  }
3919
3920  /**
3921   * See {@link #submitRegionProcedure(long)}.
3922   * @param procId the id of the open/close region procedure
3923   */
3924  public void finishRegionProcedure(long procId) {
3925    executedRegionProcedures.put(procId, procId);
3926    submittedRegionProcedures.remove(procId);
3927  }
3928
3929  public boolean isShutDown() {
3930    return shutDown;
3931  }
3932
3933  /**
3934   * Force to terminate region server when abort timeout.
3935   */
3936  private static class SystemExitWhenAbortTimeout extends TimerTask {
3937
3938    public SystemExitWhenAbortTimeout() {
3939    }
3940
3941    @Override
3942    public void run() {
3943      LOG.warn("Aborting region server timed out, terminating forcibly" +
3944          " and does not wait for any running shutdown hooks or finalizers to finish their work." +
3945          " Thread dump to stdout.");
3946      Threads.printThreadInfo(System.out, "Zombie HRegionServer");
3947      Runtime.getRuntime().halt(1);
3948    }
3949  }
3950
3951  @Override
3952  public AsyncClusterConnection getAsyncClusterConnection() {
3953    return asyncClusterConnection;
3954  }
3955
3956  @VisibleForTesting
3957  public CompactedHFilesDischarger getCompactedHFilesDischarger() {
3958    return compactedFileDischarger;
3959  }
3960}