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 2019/01/11 13:36:27 UTC

[hbase] branch branch-2.0 updated: HBASE-21704 The implementation of DistributedHBaseCluster.getServerHoldingRegion is incorrect

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

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


The following commit(s) were added to refs/heads/branch-2.0 by this push:
     new a8905cf  HBASE-21704 The implementation of DistributedHBaseCluster.getServerHoldingRegion is incorrect
a8905cf is described below

commit a8905cfffae7ef7707aad0351b35b9dbbcdb9e44
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Fri Jan 11 17:45:12 2019 +0800

    HBASE-21704 The implementation of DistributedHBaseCluster.getServerHoldingRegion is incorrect
---
 .../hadoop/hbase/DistributedHBaseCluster.java      | 45 ++++++++--------------
 .../java/org/apache/hadoop/hbase/HBaseCluster.java |  5 ++-
 2 files changed, 20 insertions(+), 30 deletions(-)

diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
index 943f2a6..1e37d24 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
@@ -31,14 +31,13 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
-import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.ServerInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
 
@@ -281,20 +280,16 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
   @Override
   public ServerName getServerHoldingRegion(TableName tn, byte[] regionName) throws IOException {
+    byte[] startKey = RegionInfo.getStartKey(regionName);
     HRegionLocation regionLoc = null;
     try (RegionLocator locator = connection.getRegionLocator(tn)) {
-      regionLoc = locator.getRegionLocation(regionName, true);
+      regionLoc = locator.getRegionLocation(startKey, true);
     }
     if (regionLoc == null) {
-      LOG.warn("Cannot find region server holding region " + Bytes.toString(regionName) +
-        ", start key [" + Bytes.toString(HRegionInfo.getStartKey(regionName)) + "]");
+      LOG.warn("Cannot find region server holding region " + Bytes.toStringBinary(regionName));
       return null;
     }
-
-    AdminProtos.AdminService.BlockingInterface client =
-        ((ClusterConnection)this.connection).getAdmin(regionLoc.getServerName());
-    ServerInfo info = ProtobufUtil.getServerInfo(null, client);
-    return ProtobufUtil.toServerName(info.getServerName());
+    return regionLoc.getServerName();
   }
 
   @Override
@@ -335,17 +330,15 @@ public class DistributedHBaseCluster extends HBaseCluster {
     //check whether current master has changed
     final ServerName initMaster = initial.getMasterName();
     if (!ServerName.isSameAddress(initMaster, current.getMasterName())) {
-      LOG.info("Restoring cluster - Initial active master : "
-              + initMaster.getHostAndPort()
-              + " has changed to : "
-              + current.getMasterName().getHostAndPort());
+      LOG.info("Restoring cluster - Initial active master : " + initMaster.getAddress() +
+        " has changed to : " + current.getMasterName().getAddress());
       // If initial master is stopped, start it, before restoring the state.
       // It will come up as a backup master, if there is already an active master.
       try {
         if (!clusterManager.isRunning(ServiceType.HBASE_MASTER,
                 initMaster.getHostname(), initMaster.getPort())) {
           LOG.info("Restoring cluster - starting initial active master at:"
-                  + initMaster.getHostAndPort());
+                  + initMaster.getAddress());
           startMaster(initMaster.getHostname(), initMaster.getPort());
         }
 
@@ -376,7 +369,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
                   backup.getHostname(),
                   backup.getPort())) {
             LOG.info("Restoring cluster - starting initial backup master: "
-                    + backup.getHostAndPort());
+                    + backup.getAddress());
             startMaster(backup.getHostname(), backup.getPort());
           }
         } catch (IOException ex) {
@@ -400,7 +393,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
       for (ServerName sn:toStart) {
         try {
           if(!clusterManager.isRunning(ServiceType.HBASE_MASTER, sn.getHostname(), sn.getPort())) {
-            LOG.info("Restoring cluster - starting initial backup master: " + sn.getHostAndPort());
+            LOG.info("Restoring cluster - starting initial backup master: " + sn.getAddress());
             startMaster(sn.getHostname(), sn.getPort());
           }
         } catch (IOException ex) {
@@ -411,7 +404,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
       for (ServerName sn:toKill) {
         try {
           if(clusterManager.isRunning(ServiceType.HBASE_MASTER, sn.getHostname(), sn.getPort())) {
-            LOG.info("Restoring cluster - stopping backup master: " + sn.getHostAndPort());
+            LOG.info("Restoring cluster - stopping backup master: " + sn.getAddress());
             stopMaster(sn);
           }
         } catch (IOException ex) {
@@ -461,11 +454,9 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
     for(ServerName sn:toStart) {
       try {
-        if (!clusterManager.isRunning(ServiceType.HBASE_REGIONSERVER,
-                sn.getHostname(),
-                sn.getPort())
-                && master.getPort() != sn.getPort()) {
-          LOG.info("Restoring cluster - starting initial region server: " + sn.getHostAndPort());
+        if (!clusterManager.isRunning(ServiceType.HBASE_REGIONSERVER, sn.getHostname(),
+          sn.getPort()) && master.getPort() != sn.getPort()) {
+          LOG.info("Restoring cluster - starting initial region server: " + sn.getAddress());
           startRegionServer(sn.getHostname(), sn.getPort());
         }
       } catch (IOException ex) {
@@ -475,11 +466,9 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
     for(ServerName sn:toKill) {
       try {
-        if (clusterManager.isRunning(ServiceType.HBASE_REGIONSERVER,
-                sn.getHostname(),
-                sn.getPort())
-                && master.getPort() != sn.getPort()){
-          LOG.info("Restoring cluster - stopping initial region server: " + sn.getHostAndPort());
+        if (clusterManager.isRunning(ServiceType.HBASE_REGIONSERVER, sn.getHostname(),
+          sn.getPort()) && master.getPort() != sn.getPort()) {
+          LOG.info("Restoring cluster - stopping initial region server: " + sn.getAddress());
           stopRegionServer(sn);
         }
       } catch (IOException ex) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
index 59a0059..35718b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -336,7 +337,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
    */
   public ServerName getServerHoldingMeta() throws IOException {
     return getServerHoldingRegion(TableName.META_TABLE_NAME,
-      HRegionInfo.FIRST_META_REGIONINFO.getRegionName());
+      RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName());
   }
 
   /**
@@ -346,7 +347,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
    * @return ServerName that hosts the region or null
    */
   public abstract ServerName getServerHoldingRegion(final TableName tn, byte[] regionName)
-  throws IOException;
+      throws IOException;
 
   /**
    * @return whether we are interacting with a distributed cluster as opposed to an