public void run()

in hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java [809:1036]


  public void run() {
    if (isStopped()) {
      LOG.info("Skipping run; stopped");
      return;
    }
    try {
      // Do pre-registration initializations; zookeeper, lease threads, etc.
      preRegistrationInitialization();
    } catch (Throwable e) {
      abort("Fatal exception during initialization", e);
    }

    try {
      if (!isStopped() && !isAborted()) {
        installShutdownHook();
        // Initialize the RegionServerCoprocessorHost now that our ephemeral
        // node was created, in case any coprocessors want to use ZooKeeper
        this.rsHost = new RegionServerCoprocessorHost(this, this.conf);

        // Try and register with the Master; tell it we are here. Break if server is stopped or
        // the clusterup flag is down or hdfs went wacky. Once registered successfully, go ahead and
        // start up all Services. Use RetryCounter to get backoff in case Master is struggling to
        // come up.
        LOG.debug("About to register with Master.");
        TraceUtil.trace(() -> {
          RetryCounterFactory rcf =
            new RetryCounterFactory(Integer.MAX_VALUE, this.sleeper.getPeriod(), 1000 * 60 * 5);
          RetryCounter rc = rcf.create();
          while (keepLooping()) {
            RegionServerStartupResponse w = reportForDuty();
            if (w == null) {
              long sleepTime = rc.getBackoffTimeAndIncrementAttempts();
              LOG.warn("reportForDuty failed; sleeping {} ms and then retrying.", sleepTime);
              this.sleeper.sleep(sleepTime);
            } else {
              handleReportForDutyResponse(w);
              break;
            }
          }
        }, "HRegionServer.registerWithMaster");
      }

      if (!isStopped() && isHealthy()) {
        TraceUtil.trace(() -> {
          // start the snapshot handler and other procedure handlers,
          // since the server is ready to run
          if (this.rspmHost != null) {
            this.rspmHost.start();
          }
          // Start the Quota Manager
          if (this.rsQuotaManager != null) {
            rsQuotaManager.start(getRpcServer().getScheduler());
          }
          if (this.rsSpaceQuotaManager != null) {
            this.rsSpaceQuotaManager.start();
          }
        }, "HRegionServer.startup");
      }

      // We registered with the Master. Go into run mode.
      long lastMsg = EnvironmentEdgeManager.currentTime();
      long oldRequestCount = -1;
      // The main run loop.
      while (!isStopped() && isHealthy()) {
        if (!isClusterUp()) {
          if (onlineRegions.isEmpty()) {
            stop("Exiting; cluster shutdown set and not carrying any regions");
          } else if (!this.stopping) {
            this.stopping = true;
            LOG.info("Closing user regions");
            closeUserRegions(isAborted());
          } else {
            boolean allUserRegionsOffline = areAllUserRegionsOffline();
            if (allUserRegionsOffline) {
              // Set stopped if no more write requests tp meta tables
              // since last time we went around the loop. Any open
              // meta regions will be closed on our way out.
              if (oldRequestCount == getWriteRequestCount()) {
                stop("Stopped; only catalog regions remaining online");
                break;
              }
              oldRequestCount = getWriteRequestCount();
            } else {
              // Make sure all regions have been closed -- some regions may
              // have not got it because we were splitting at the time of
              // the call to closeUserRegions.
              closeUserRegions(this.abortRequested.get());
            }
            LOG.debug("Waiting on " + getOnlineRegionsAsPrintableString());
          }
        }
        long now = EnvironmentEdgeManager.currentTime();
        if ((now - lastMsg) >= msgInterval) {
          tryRegionServerReport(lastMsg, now);
          lastMsg = EnvironmentEdgeManager.currentTime();
        }
        if (!isStopped() && !isAborted()) {
          this.sleeper.sleep();
        }
      } // for
    } catch (Throwable t) {
      if (!rpcServices.checkOOME(t)) {
        String prefix = t instanceof YouAreDeadException ? "" : "Unhandled: ";
        abort(prefix + t.getMessage(), t);
      }
    }

    final Span span = TraceUtil.createSpan("HRegionServer exiting main loop");
    try (Scope ignored = span.makeCurrent()) {
      if (this.leaseManager != null) {
        this.leaseManager.closeAfterLeasesExpire();
      }
      if (this.splitLogWorker != null) {
        splitLogWorker.stop();
      }
      stopInfoServer();
      // Send cache a shutdown.
      if (blockCache != null) {
        blockCache.shutdown();
      }
      if (mobFileCache != null) {
        mobFileCache.shutdown();
      }

      // Send interrupts to wake up threads if sleeping so they notice shutdown.
      // TODO: Should we check they are alive? If OOME could have exited already
      if (this.hMemManager != null) {
        this.hMemManager.stop();
      }
      if (this.cacheFlusher != null) {
        this.cacheFlusher.interruptIfNecessary();
      }
      if (this.compactSplitThread != null) {
        this.compactSplitThread.interruptIfNecessary();
      }

      // Stop the snapshot and other procedure handlers, forcefully killing all running tasks
      if (rspmHost != null) {
        rspmHost.stop(this.abortRequested.get() || this.killed);
      }

      if (this.killed) {
        // Just skip out w/o closing regions. Used when testing.
      } else if (abortRequested.get()) {
        if (this.dataFsOk) {
          closeUserRegions(abortRequested.get()); // Don't leave any open file handles
        }
        LOG.info("aborting server " + this.serverName);
      } else {
        closeUserRegions(abortRequested.get());
        LOG.info("stopping server " + this.serverName);
      }
      regionReplicationBufferManager.stop();
      closeClusterConnection();
      // Closing the compactSplit thread before closing meta regions
      if (!this.killed && containsMetaTableRegions()) {
        if (!abortRequested.get() || this.dataFsOk) {
          if (this.compactSplitThread != null) {
            this.compactSplitThread.join();
            this.compactSplitThread = null;
          }
          closeMetaTableRegions(abortRequested.get());
        }
      }

      if (!this.killed && this.dataFsOk) {
        waitOnAllRegionsToClose(abortRequested.get());
        LOG.info("stopping server " + this.serverName + "; all regions closed.");
      }

      // Stop the quota manager
      if (rsQuotaManager != null) {
        rsQuotaManager.stop();
      }
      if (rsSpaceQuotaManager != null) {
        rsSpaceQuotaManager.stop();
        rsSpaceQuotaManager = null;
      }

      // flag may be changed when closing regions throws exception.
      if (this.dataFsOk) {
        shutdownWAL(!abortRequested.get());
      }

      // Make sure the proxy is down.
      if (this.rssStub != null) {
        this.rssStub = null;
      }
      if (this.lockStub != null) {
        this.lockStub = null;
      }
      if (this.rpcClient != null) {
        this.rpcClient.close();
      }
      if (this.leaseManager != null) {
        this.leaseManager.close();
      }
      if (this.pauseMonitor != null) {
        this.pauseMonitor.stop();
      }

      if (!killed) {
        stopServiceThreads();
      }

      if (this.rpcServices != null) {
        this.rpcServices.stop();
      }

      try {
        deleteMyEphemeralNode();
      } catch (KeeperException.NoNodeException nn) {
        // pass
      } catch (KeeperException e) {
        LOG.warn("Failed deleting my ephemeral node", e);
      }
      // We may have failed to delete the znode at the previous step, but
      // we delete the file anyway: a second attempt to delete the znode is likely to fail again.
      ZNodeClearer.deleteMyEphemeralNodeOnDisk();

      closeZooKeeper();
      closeTableDescriptors();
      LOG.info("Exiting; stopping=" + this.serverName + "; zookeeper connection closed.");
      span.setStatus(StatusCode.OK);
    } finally {
      span.end();
    }
  }