You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2023/10/23 09:16:19 UTC

[hbase] branch branch-2 updated: HBASE-28146: Make ServerManager rsAdmins map thread safe (#5461)

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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 1641a4abc13 HBASE-28146: Make ServerManager rsAdmins map thread safe (#5461)
1641a4abc13 is described below

commit 1641a4abc13c7ad4da216a115107fc772d24d543
Author: Ray Mattingly <rm...@gmail.com>
AuthorDate: Mon Oct 23 05:16:14 2023 -0400

    HBASE-28146: Make ServerManager rsAdmins map thread safe (#5461)
    
    Co-authored-by: Ray Mattingly <rm...@hubspot.com>
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Bryan Beaudreault <bb...@apache.org>
---
 .../apache/hadoop/hbase/master/ServerManager.java  | 26 +++++-----------------
 1 file changed, 6 insertions(+), 20 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 ed37fd95444..196a1a582ed 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
@@ -23,7 +23,6 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -124,12 +123,6 @@ public class ServerManager {
   private final ConcurrentNavigableMap<ServerName, ServerMetrics> onlineServers =
     new ConcurrentSkipListMap<>();
 
-  /**
-   * Map of admin interfaces per registered regionserver; these interfaces we use to control
-   * regionservers out on the cluster
-   */
-  private final Map<ServerName, AdminService.BlockingInterface> rsAdmins = new HashMap<>();
-
   /** List of region servers that should not get any more new regions. */
   private final ArrayList<ServerName> drainingServers = new ArrayList<>();
 
@@ -402,7 +395,6 @@ public class ServerManager {
   void recordNewServerWithLock(final ServerName serverName, final ServerMetrics sl) {
     LOG.info("Registering regionserver=" + serverName);
     this.onlineServers.put(serverName, sl);
-    this.rsAdmins.remove(serverName);
   }
 
   public RegionStoreSequenceIds getLastFlushedSequenceId(byte[] encodedRegionName) {
@@ -604,7 +596,6 @@ public class ServerManager {
         LOG.trace("Expiration of {} but server not online", sn);
       }
     }
-    this.rsAdmins.remove(sn);
   }
 
   /*
@@ -716,18 +707,13 @@ public class ServerManager {
    * @throws RetriesExhaustedException wrapping a ConnectException if failed
    */
   public AdminService.BlockingInterface getRsAdmin(final ServerName sn) throws IOException {
-    AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
-    if (admin == null) {
-      LOG.debug("New admin connection to " + sn.toString());
-      if (sn.equals(master.getServerName()) && master instanceof HRegionServer) {
-        // A master is also a region server now, see HBASE-10569 for details
-        admin = ((HRegionServer) master).getRSRpcServices();
-      } else {
-        admin = this.connection.getAdmin(sn);
-      }
-      this.rsAdmins.put(sn, admin);
+    LOG.debug("New admin connection to {}", sn);
+    if (sn.equals(master.getServerName()) && master instanceof HRegionServer) {
+      // A master is also a region server now, see HBASE-10569 for details
+      return ((HRegionServer) master).getRSRpcServices();
+    } else {
+      return this.connection.getAdmin(sn);
     }
-    return admin;
   }
 
   /**