You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ta...@apache.org on 2021/08/27 16:42:48 UTC

[hbase] 05/18: HBASE-26128 Backport HBASE-25454 "Add trace support for connection re… (#3561)

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

taklwu pushed a commit to branch HBASE-25853
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit c3d577a3b2afc6024a9d02d6f6ef9fae3bed53b1
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Thu Aug 12 12:38:53 2021 -0700

    HBASE-26128 Backport HBASE-25454 "Add trace support for connection re… (#3561)
    
    5/17 commits of HBASE-22120, original commit dcb78bd4bda4a4ae13d863df8aec266031e5bc93 and merged conflicts after rebasing on HBASE-26150 with commit 63d4970de451bf234f2ddbda949995b1420e525b
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../client/AbstractRpcBasedConnectionRegistry.java | 52 +++++++++++++---------
 .../hadoop/hbase/client/ZKConnectionRegistry.java  | 36 ++++++++-------
 2 files changed, 53 insertions(+), 35 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java
index 7d946aa..54138d3 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AbstractRpcBasedConnectionRegistry.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.trace.TraceUtil.trace;
+import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFuture;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import com.google.errorprone.annotations.RestrictedApi;
@@ -237,38 +239,48 @@ abstract class AbstractRpcBasedConnectionRegistry implements ConnectionRegistry
 
   @Override
   public CompletableFuture<RegionLocations> getMetaRegionLocations() {
-    return this
-      .<GetMetaRegionLocationsResponse> call((c, s, d) -> s.getMetaRegionLocations(c,
-        GetMetaRegionLocationsRequest.getDefaultInstance(), d), r -> r.getMetaLocationsCount() != 0,
+    return tracedFuture(
+      () -> this
+        .<GetMetaRegionLocationsResponse> call(
+          (c, s, d) -> s.getMetaRegionLocations(c,
+            GetMetaRegionLocationsRequest.getDefaultInstance(), d),
+          r -> r.getMetaLocationsCount() != 0,
         "getMetaLocationsCount")
-      .thenApply(AbstractRpcBasedConnectionRegistry::transformMetaRegionLocations);
+        .thenApply(AbstractRpcBasedConnectionRegistry::transformMetaRegionLocations),
+      getClass().getSimpleName() + ".getMetaRegionLocations");
   }
 
   @Override
   public CompletableFuture<String> getClusterId() {
-    return this
-      .<GetClusterIdResponse> call(
-        (c, s, d) -> s.getClusterId(c, GetClusterIdRequest.getDefaultInstance(), d),
-        GetClusterIdResponse::hasClusterId, "getClusterId()")
-      .thenApply(GetClusterIdResponse::getClusterId);
+    return tracedFuture(
+      () -> this
+        .<GetClusterIdResponse> call(
+          (c, s, d) -> s.getClusterId(c, GetClusterIdRequest.getDefaultInstance(), d),
+          GetClusterIdResponse::hasClusterId, "getClusterId()")
+        .thenApply(GetClusterIdResponse::getClusterId),
+      getClass().getSimpleName() + ".getClusterId");
   }
 
   @Override
   public CompletableFuture<ServerName> getActiveMaster() {
-    return this
-      .<GetActiveMasterResponse> call(
-        (c, s, d) -> s.getActiveMaster(c, GetActiveMasterRequest.getDefaultInstance(), d),
-        GetActiveMasterResponse::hasServerName, "getActiveMaster()")
-      .thenApply(resp -> ProtobufUtil.toServerName(resp.getServerName()));
+    return tracedFuture(
+      () -> this
+        .<GetActiveMasterResponse>call(
+          (c, s, d) -> s.getActiveMaster(c, GetActiveMasterRequest.getDefaultInstance(), d),
+          GetActiveMasterResponse::hasServerName, "getActiveMaster()")
+        .thenApply(resp -> ProtobufUtil.toServerName(resp.getServerName())),
+      getClass().getSimpleName() + ".getActiveMaster");
   }
 
   @Override
   public void close() {
-    if (registryEndpointRefresher != null) {
-      registryEndpointRefresher.stop();
-    }
-    if (rpcClient != null) {
-      rpcClient.close();
-    }
+    trace(() -> {
+      if (registryEndpointRefresher != null) {
+        registryEndpointRefresher.stop();
+      }
+      if (rpcClient != null) {
+        rpcClient.close();
+      }
+    }, getClass().getSimpleName() + ".close");
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
index 35107ca..6e94afe 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
@@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.client.RegionInfoBuilder.FIRST_META_REGION
 import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForDefaultReplica;
 import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForReplica;
 import static org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.lengthOfPBMagic;
+import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFuture;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData;
 
@@ -99,7 +100,9 @@ class ZKConnectionRegistry implements ConnectionRegistry {
 
   @Override
   public CompletableFuture<String> getClusterId() {
-    return getAndConvert(znodePaths.clusterIdZNode, ZKConnectionRegistry::getClusterId);
+    return tracedFuture(
+      () -> getAndConvert(znodePaths.clusterIdZNode, ZKConnectionRegistry::getClusterId),
+      "ZKConnectionRegistry.getClusterId");
   }
 
   ReadOnlyZKClient getZKClient() {
@@ -204,19 +207,20 @@ class ZKConnectionRegistry implements ConnectionRegistry {
 
   @Override
   public CompletableFuture<RegionLocations> getMetaRegionLocations() {
-    CompletableFuture<RegionLocations> future = new CompletableFuture<>();
-    addListener(
-      zk.list(znodePaths.baseZNode)
-        .thenApply(children -> children.stream()
+    return tracedFuture(() -> {
+      CompletableFuture<RegionLocations> future = new CompletableFuture<>();
+      addListener(
+        zk.list(znodePaths.baseZNode).thenApply(children -> children.stream()
           .filter(c -> this.znodePaths.isMetaZNodePrefix(c)).collect(Collectors.toList())),
-      (metaReplicaZNodes, error) -> {
-        if (error != null) {
-          future.completeExceptionally(error);
-          return;
-        }
-        getMetaRegionLocation(future, metaReplicaZNodes);
-      });
-    return future;
+        (metaReplicaZNodes, error) -> {
+          if (error != null) {
+            future.completeExceptionally(error);
+            return;
+          }
+          getMetaRegionLocation(future, metaReplicaZNodes);
+        });
+      return future;
+    }, "ZKConnectionRegistry.getMetaRegionLocations");
   }
 
   private static ZooKeeperProtos.Master getMasterProto(byte[] data) throws IOException {
@@ -230,7 +234,8 @@ class ZKConnectionRegistry implements ConnectionRegistry {
 
   @Override
   public CompletableFuture<ServerName> getActiveMaster() {
-    return getAndConvert(znodePaths.masterAddressZNode, ZKConnectionRegistry::getMasterProto)
+    return tracedFuture(
+      () -> getAndConvert(znodePaths.masterAddressZNode, ZKConnectionRegistry::getMasterProto)
         .thenApply(proto -> {
           if (proto == null) {
             return null;
@@ -238,7 +243,8 @@ class ZKConnectionRegistry implements ConnectionRegistry {
           HBaseProtos.ServerName snProto = proto.getMaster();
           return ServerName.valueOf(snProto.getHostName(), snProto.getPort(),
             snProto.getStartCode());
-        });
+        }),
+      "ZKConnectionRegistry.getActiveMaster");
   }
 
   @Override