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