[hbase] 17/18: HBASE-26140 Backport HBASE-25778 "The tracinig implementation for AsyncConnectionImpl.getHbck is incorrect" to branch-2 (#3631)

2021-09-01 Thread taklwu
This is an automated email from the ASF dual-hosted git repository.

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

commit ab431fc8a9b106091f703eeea3a4f71de16b83e7
Author: Tak Lon (Stephen) Wu 
AuthorDate: Thu Aug 26 11:29:30 2021 -0700

HBASE-26140 Backport HBASE-25778 "The tracinig implementation for 
AsyncConnectionImpl.getHbck is incorrect" to branch-2 (#3631)

17/17 commits of HBASE-22120, original commit 
f36e1539648bbaee84c626fd54d1605baebf3c5a

Co-authored-by: Duo Zhang 

Signed-off-by: Peter Somogyi 
---
 .../hadoop/hbase/client/AsyncConnectionImpl.java   |  37 +++
 .../hbase/client/TestAsyncConnectionTracing.java   | 112 +
 .../org/apache/hadoop/hbase/trace/TraceUtil.java   |  14 +++
 3 files changed, 145 insertions(+), 18 deletions(-)

diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 98e7825..76b1208 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -28,7 +28,6 @@ import static 
org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLE
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -37,6 +36,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
@@ -370,6 +370,15 @@ class AsyncConnectionImpl implements AsyncConnection {
   RETRY_TIMER);
   }
 
+  private Hbck getHbckInternal(ServerName masterServer) {
+Span.current().setAttribute(TraceUtil.SERVER_NAME_KEY, 
masterServer.getServerName());
+// we will not create a new connection when creating a new protobuf stub, 
and for hbck there
+// will be no performance consideration, so for simplification we will 
create a new stub every
+// time instead of caching the stub here.
+return new HBaseHbck(MasterProtos.HbckService.newBlockingStub(
+  rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout)), 
rpcControllerFactory);
+  }
+
   @Override
   public CompletableFuture getHbck() {
 return TraceUtil.tracedFuture(() -> {
@@ -378,11 +387,7 @@ class AsyncConnectionImpl implements AsyncConnection {
 if (error != null) {
   future.completeExceptionally(error);
 } else {
-  try {
-future.complete(getHbck(sn));
-  } catch (IOException e) {
-future.completeExceptionally(e);
-  }
+  future.complete(getHbckInternal(sn));
 }
   });
   return future;
@@ -390,18 +395,14 @@ class AsyncConnectionImpl implements AsyncConnection {
   }
 
   @Override
-  public Hbck getHbck(ServerName masterServer) throws IOException {
-Span span = TraceUtil.createSpan("AsyncConnection.getHbck")
-  .setAttribute(TraceUtil.SERVER_NAME_KEY, masterServer.getServerName());
-try (Scope scope = span.makeCurrent()) {
-  // we will not create a new connection when creating a new protobuf 
stub, and for hbck there
-  // will be no performance consideration, so for simplification we will 
create a new stub every
-  // time instead of caching the stub here.
-  return new HBaseHbck(
-MasterProtos.HbckService
-  .newBlockingStub(rpcClient.createBlockingRpcChannel(masterServer, 
user, rpcTimeout)),
-rpcControllerFactory);
-}
+  public Hbck getHbck(ServerName masterServer) {
+return TraceUtil.trace(new Supplier() {
+
+  @Override
+  public Hbck get() {
+return getHbckInternal(masterServer);
+  }
+}, "AsyncConnection.getHbck");
   }
 
   @Override
diff --git 
a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java
 
b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java
new file mode 100644
index 000..fec5f6d
--- /dev/null
+++ 
b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. 

[hbase] 17/18: HBASE-26140 Backport HBASE-25778 "The tracinig implementation for AsyncConnectionImpl.getHbck is incorrect" to branch-2 (#3631)

2021-08-27 Thread taklwu
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 275de8d024a3e86e5ee18c677a2dd3668fd6b845
Author: Tak Lon (Stephen) Wu 
AuthorDate: Thu Aug 26 11:29:30 2021 -0700

HBASE-26140 Backport HBASE-25778 "The tracinig implementation for 
AsyncConnectionImpl.getHbck is incorrect" to branch-2 (#3631)

17/17 commits of HBASE-22120, original commit 
f36e1539648bbaee84c626fd54d1605baebf3c5a

Co-authored-by: Duo Zhang 

Signed-off-by: Peter Somogyi 
---
 .../hadoop/hbase/client/AsyncConnectionImpl.java   |  37 +++
 .../hbase/client/TestAsyncConnectionTracing.java   | 112 +
 .../org/apache/hadoop/hbase/trace/TraceUtil.java   |  14 +++
 3 files changed, 145 insertions(+), 18 deletions(-)

diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 98e7825..76b1208 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -28,7 +28,6 @@ import static 
org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLE
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.util.Optional;
 import java.util.concurrent.CompletableFuture;
@@ -37,6 +36,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
@@ -370,6 +370,15 @@ class AsyncConnectionImpl implements AsyncConnection {
   RETRY_TIMER);
   }
 
+  private Hbck getHbckInternal(ServerName masterServer) {
+Span.current().setAttribute(TraceUtil.SERVER_NAME_KEY, 
masterServer.getServerName());
+// we will not create a new connection when creating a new protobuf stub, 
and for hbck there
+// will be no performance consideration, so for simplification we will 
create a new stub every
+// time instead of caching the stub here.
+return new HBaseHbck(MasterProtos.HbckService.newBlockingStub(
+  rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout)), 
rpcControllerFactory);
+  }
+
   @Override
   public CompletableFuture getHbck() {
 return TraceUtil.tracedFuture(() -> {
@@ -378,11 +387,7 @@ class AsyncConnectionImpl implements AsyncConnection {
 if (error != null) {
   future.completeExceptionally(error);
 } else {
-  try {
-future.complete(getHbck(sn));
-  } catch (IOException e) {
-future.completeExceptionally(e);
-  }
+  future.complete(getHbckInternal(sn));
 }
   });
   return future;
@@ -390,18 +395,14 @@ class AsyncConnectionImpl implements AsyncConnection {
   }
 
   @Override
-  public Hbck getHbck(ServerName masterServer) throws IOException {
-Span span = TraceUtil.createSpan("AsyncConnection.getHbck")
-  .setAttribute(TraceUtil.SERVER_NAME_KEY, masterServer.getServerName());
-try (Scope scope = span.makeCurrent()) {
-  // we will not create a new connection when creating a new protobuf 
stub, and for hbck there
-  // will be no performance consideration, so for simplification we will 
create a new stub every
-  // time instead of caching the stub here.
-  return new HBaseHbck(
-MasterProtos.HbckService
-  .newBlockingStub(rpcClient.createBlockingRpcChannel(masterServer, 
user, rpcTimeout)),
-rpcControllerFactory);
-}
+  public Hbck getHbck(ServerName masterServer) {
+return TraceUtil.trace(new Supplier() {
+
+  @Override
+  public Hbck get() {
+return getHbckInternal(masterServer);
+  }
+}, "AsyncConnection.getHbck");
   }
 
   @Override
diff --git 
a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java
 
b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java
new file mode 100644
index 000..fec5f6d
--- /dev/null
+++ 
b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the