You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bh...@apache.org on 2021/03/26 00:12:36 UTC

[hbase] branch branch-1 updated: HBASE-25032 Wait for region server to become online before adding it to online servers in Master (#2770)

This is an automated email from the ASF dual-hosted git repository.

bharathv pushed a commit to branch branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-1 by this push:
     new 7652bef  HBASE-25032 Wait for region server to become online before adding it to online servers in Master (#2770)
7652bef is described below

commit 7652bef985b52624f0903a44a6b6d08759b97cd7
Author: caroliney14 <ca...@berkeley.edu>
AuthorDate: Thu Mar 25 17:11:22 2021 -0700

    HBASE-25032 Wait for region server to become online before adding it to online servers in Master (#2770)
    
    Signed-off-by: Bharath Vissapragada <bh...@apache.org>
    Signed-off-by: Michael Stack <st...@apache.org>
---
 .../apache/hadoop/hbase/master/ServerManager.java  |  8 ++----
 .../hadoop/hbase/regionserver/HRegionServer.java   | 33 ++++++++++++----------
 .../hbase/master/TestGetReplicationLoad.java       |  3 +-
 .../hadoop/hbase/master/TestMasterMetrics.java     |  6 ----
 .../TestCompactionInDeadRegionServer.java          |  3 +-
 5 files changed, 24 insertions(+), 29 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
index 5a48ce8..639900e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java
@@ -255,7 +255,8 @@ public class ServerManager {
   }
 
   /**
-   * Let the server manager know a new regionserver has come online
+   * Let the server manager know a regionserver is requesting configurations.
+   * Regionserver will not be added here, but in its first report.
    * @param request the startup request
    * @param ia the InetAddress from which request is received
    * @return The ServerName we know this server as.
@@ -277,10 +278,6 @@ public class ServerManager {
       request.getServerStartCode());
     checkClockSkew(sn, request.getServerCurrentTime());
     checkIsDead(sn, "STARTUP");
-    if (!checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
-      LOG.warn("THIS SHOULD NOT HAPPEN, RegionServerStartup"
-        + " could not record the server: " + sn);
-    }
     return sn;
   }
 
@@ -343,7 +340,6 @@ public class ServerManager {
     if (null == this.onlineServers.replace(sn, sl)) {
       // Already have this host+port combo and its just different start code?
       // Just let the server in. Presume master joining a running cluster.
-      // recordNewServer is what happens at the end of reportServerStartup.
       // The only thing we are skipping is passing back to the regionserver
       // the ServerName to use. Here we presume a master has already done
       // that so we'll press on with whatever it gave us for ServerName.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 5f5c9c5..a116f5e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1027,9 +1027,9 @@ public class HRegionServer extends HasThread implements
         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.
+      // Get configurations from the Master. Break if server is stopped or
+      // the clusterup flag is down or hdfs went wacky. Then start up all Services.
+      // Use RetryCounter to get backoff in case Master is struggling to come up.
       RetryCounterFactory rcf = new RetryCounterFactory(Integer.MAX_VALUE,
           this.sleeper.getPeriod(), 1000 * 60 * 5);
       RetryCounter rc = rcf.create();
@@ -1056,7 +1056,7 @@ public class HRegionServer extends HasThread implements
         rsQuotaManager.start(getRpcServer().getScheduler());
       }
 
-      // We registered with the Master.  Go into run mode.
+      // Run mode.
       long lastMsg = System.currentTimeMillis();
       long oldRequestCount = -1;
       // The main run loop.
@@ -1090,7 +1090,14 @@ public class HRegionServer extends HasThread implements
         }
         long now = System.currentTimeMillis();
         if ((now - lastMsg) >= msgInterval) {
-          tryRegionServerReport(lastMsg, now);
+          // Register with the Master now that our setup is complete.
+          if (tryRegionServerReport(lastMsg, now) && !online.get()) {
+            // Wake up anyone waiting for this server to online
+            synchronized (online) {
+              online.set(true);
+              online.notifyAll();
+            }
+          }
           lastMsg = System.currentTimeMillis();
         }
         if (!isStopped() && !isAborted()) {
@@ -1282,12 +1289,12 @@ public class HRegionServer extends HasThread implements
   }
 
   @InterfaceAudience.Private
-  protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
+  protected boolean tryRegionServerReport(long reportStartTime, long reportEndTime)
   throws IOException {
     RegionServerStatusService.BlockingInterface rss = rssStub;
     if (rss == null) {
       // the current server could be stopping.
-      return;
+      return false;
     }
     ClusterStatusProtos.ServerLoad sl = buildServerLoad(reportStartTime, reportEndTime);
     try {
@@ -1307,7 +1314,9 @@ public class HRegionServer extends HasThread implements
       // Couldn't connect to the master, get location from zk and reconnect
       // Method blocks until new master is found or we are stopped
       createRegionServerStatusStub(true);
+      return false;
     }
+    return true;
   }
 
   ClusterStatusProtos.ServerLoad buildServerLoad(long reportStartTime, long reportEndTime)
@@ -1571,11 +1580,6 @@ public class HRegionServer extends HasThread implements
         ", sessionid=0x" +
         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()));
 
-      // Wake up anyone waiting for this server to online
-      synchronized (online) {
-        online.set(true);
-        online.notifyAll();
-      }
     } catch (Throwable e) {
       stop("Failed initialization");
       throw convertThrowableToIOE(cleanup(e, "Failed init"),
@@ -2554,10 +2558,9 @@ public class HRegionServer extends HasThread implements
   }
 
   /*
-   * Let the master know we're here Run initialization using parameters passed
-   * us by the master.
+   * Run initialization using parameters passed us by the master.
    * @return A Map of key/value configurations we got from the Master else
-   * null if we failed to register.
+   * null if we failed during report.
    * @throws IOException
    */
   private RegionServerStartupResponse reportForDuty() throws IOException {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetReplicationLoad.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetReplicationLoad.java
index bad92b8..6ce8d76 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetReplicationLoad.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestGetReplicationLoad.java
@@ -60,8 +60,9 @@ public class TestGetReplicationLoad {
     }
 
     @Override
-    protected void tryRegionServerReport(long reportStartTime, long reportEndTime) {
+    protected boolean tryRegionServerReport(long reportStartTime, long reportEndTime) {
       // do nothing
+      return true;
     }
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
index 554e36e..ead3a58 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterMetrics.java
@@ -54,12 +54,6 @@ public class TestMasterMetrics {
         KeeperException, InterruptedException {
       super(conf, cp);
     }
-
-    @Override
-    protected void tryRegionServerReport(
-        long reportStartTime, long reportEndTime) {
-      // do nothing
-    }
   }
 
   @BeforeClass
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
index 879823b..32ba93f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactionInDeadRegionServer.java
@@ -74,13 +74,14 @@ public class TestCompactionInDeadRegionServer {
     }
 
     @Override
-    protected void tryRegionServerReport(long reportStartTime, long reportEndTime)
+    protected boolean tryRegionServerReport(long reportStartTime, long reportEndTime)
         throws IOException {
       try {
         super.tryRegionServerReport(reportStartTime, reportEndTime);
       } catch (YouAreDeadException e) {
         // ignore, do not abort
       }
+      return true;
     }
   }