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