This is an automated email from the ASF dual-hosted git repository. zhangduo pushed a commit to branch HBASE-21512 in repository https://gitbox.apache.org/repos/asf/hbase.git
commit 1ba380d561624c77d08399a65bf7384492c314b1 Author: Duo Zhang <zhang...@apache.org> AuthorDate: Thu Jan 10 11:47:41 2019 +0800 HBASE-21537 Rewrite ServerManager.closeRegionSilentlyAndWait to use AsyncClusterConnection Signed-off-by: Michael Stack <st...@apache.org> --- .../hadoop/hbase/master/MasterMetaBootstrap.java | 2 +- .../apache/hadoop/hbase/master/ServerManager.java | 41 +++++++++++++--------- .../apache/hadoop/hbase/util/HBaseFsckRepair.java | 22 +++++++----- 3 files changed, 38 insertions(+), 27 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java index e57817e..6e38bdd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java @@ -101,7 +101,7 @@ class MasterMetaBootstrap { RegionState r = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId); LOG.info("Closing excess replica of meta region " + r.getRegion()); // send a close and wait for a max of 30 seconds - ServerManager.closeRegionSilentlyAndWait(master.getClusterConnection(), + ServerManager.closeRegionSilentlyAndWait(master.getAsyncClusterConnection(), r.getServerName(), r.getRegion(), 30000); ZKUtil.deleteNode(zooKeeper, zooKeeper.getZNodePaths().getZNodeForReplica(replicaId)); } 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 a8d5e21..f98899e 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 @@ -48,13 +48,15 @@ import org.apache.hadoop.hbase.ServerMetrics; import org.apache.hadoop.hbase.ServerMetricsBuilder; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.YouAreDeadException; -import org.apache.hadoop.hbase.client.ClusterConnection; +import org.apache.hadoop.hbase.client.AsyncClusterConnection; +import org.apache.hadoop.hbase.client.AsyncRegionServerAdmin; import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.master.assignment.RegionStates; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.yetus.audience.InterfaceAudience; @@ -67,6 +69,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.ByteString; import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId; @@ -660,35 +663,39 @@ public class ServerManager { } /** - * Contacts a region server and waits up to timeout ms - * to close the region. This bypasses the active hmaster. + * Contacts a region server and waits up to timeout ms to close the region. This bypasses the + * active hmaster. */ - public static void closeRegionSilentlyAndWait(ClusterConnection connection, - ServerName server, RegionInfo region, long timeout) throws IOException, InterruptedException { - AdminService.BlockingInterface rs = connection.getAdmin(server); - HBaseRpcController controller = connection.getRpcControllerFactory().newController(); + public static void closeRegionSilentlyAndWait(AsyncClusterConnection connection, + ServerName server, RegionInfo region, long timeout) throws IOException, InterruptedException { + AsyncRegionServerAdmin admin = connection.getRegionServerAdmin(server); try { - ProtobufUtil.closeRegion(controller, rs, server, region.getRegionName()); + FutureUtils.get( + admin.closeRegion(ProtobufUtil.buildCloseRegionRequest(server, region.getRegionName()))); } catch (IOException e) { LOG.warn("Exception when closing region: " + region.getRegionNameAsString(), e); } long expiration = timeout + System.currentTimeMillis(); while (System.currentTimeMillis() < expiration) { - controller.reset(); try { - RegionInfo rsRegion = - ProtobufUtil.getRegionInfo(controller, rs, region.getRegionName()); - if (rsRegion == null) return; + RegionInfo rsRegion = ProtobufUtil.toRegionInfo(FutureUtils + .get( + admin.getRegionInfo(RequestConverter.buildGetRegionInfoRequest(region.getRegionName()))) + .getRegionInfo()); + if (rsRegion == null) { + return; + } } catch (IOException ioe) { - if (ioe instanceof NotServingRegionException) // no need to retry again + if (ioe instanceof NotServingRegionException) { + // no need to retry again return; - LOG.warn("Exception when retrieving regioninfo from: " - + region.getRegionNameAsString(), ioe); + } + LOG.warn("Exception when retrieving regioninfo from: " + region.getRegionNameAsString(), + ioe); } Thread.sleep(1000); } - throw new IOException("Region " + region + " failed to close within" - + " timeout " + timeout); + throw new IOException("Region " + region + " failed to close within" + " timeout " + timeout); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java index ec7f717..121d06c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsckRepair.java @@ -31,7 +31,9 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.AsyncClusterConnection; import org.apache.hadoop.hbase.client.ClusterConnection; +import org.apache.hadoop.hbase.client.ClusterConnectionFactory; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.Put; @@ -41,6 +43,7 @@ import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.ServerManager; import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.security.User; import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; @@ -143,16 +146,17 @@ public class HBaseFsckRepair { } /** - * Contacts a region server and waits up to hbase.hbck.close.timeout ms - * (default 120s) to close the region. This bypasses the active hmaster. + * Contacts a region server and waits up to hbase.hbck.close.timeout ms (default 120s) to close + * the region. This bypasses the active hmaster. */ - @SuppressWarnings("deprecation") - public static void closeRegionSilentlyAndWait(Connection connection, - ServerName server, RegionInfo region) throws IOException, InterruptedException { - long timeout = connection.getConfiguration() - .getLong("hbase.hbck.close.timeout", 120000); - ServerManager.closeRegionSilentlyAndWait((ClusterConnection)connection, server, - region, timeout); + public static void closeRegionSilentlyAndWait(Connection connection, ServerName server, + RegionInfo region) throws IOException, InterruptedException { + long timeout = connection.getConfiguration().getLong("hbase.hbck.close.timeout", 120000); + // this is a bit ugly but it is only used in the old hbck and tests, so I think it is fine. + try (AsyncClusterConnection asyncConn = ClusterConnectionFactory + .createAsyncClusterConnection(connection.getConfiguration(), null, User.getCurrent())) { + ServerManager.closeRegionSilentlyAndWait(asyncConn, server, region, timeout); + } } /**