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/09/01 22:29:45 UTC

[hbase] branch branch-2 updated (008ffd2 -> 625093f)

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

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


    from 008ffd2  HBASE-26205 Use specified cluster conf for UserProvider in TableMRUtil#initCredentialsForCluster (#3592)
     new 665305c  HBASE-26124 Backport HBASE-25373 "Remove HTrace completely in code base and try to make use of OpenTelemetry" to branch-2 (#3529)
     new 4dfc59c  HBASE-26125 Backport HBASE-25401 "Add trace support for async call in rpc client" to branch-2 (#3543)
     new af46013  HBASE-26126 Backport HBASE-25424 "Find a way to config OpenTelemetry tracing without directly depending on opentelemetry-sdk" to branch-2 (#3552)
     new 58b5c00  HBASE-26127 Backport HBASE-23898 "Add trace support for simple apis i… (#3556)
     new 7ecf54f  HBASE-26128 Backport HBASE-25454 "Add trace support for connection re… (#3561)
     new 83a2738  HBASE-26129 Backport HBASE-25481 "Add host and port attribute when tracing rpc call at client side" to branch-2 (#3589)
     new 321dcd4  HBASE-26130 Backport HBASE-25455 "Add trace support for HRegion read/… (#3594)
     new 9cce94a  HBASE-26131 Backport HBASE-25484 "Add trace support for WAL sync" to branch-2 (#3597)
     new 8b7450f  HBASE-26132 Backport HBASE-25535 "Set span kind to CLIENT in AbstractRpcClient" to branch-2 (#3607)
     new d0a53e3  HBASE-26133 Backport HBASE-25591 "Upgrade opentelemetry to 0.17.1" to branch-2 (#3608)
     new a2b8f7a  HBASE-26134 Backport HBASE-25617 "Revisit the span names" to branch-2 (#3618)
     new 8f16010  HBASE-26135 Backport HBASE-25616 "Upgrade opentelemetry to 1.0.0" to branch-2 (#3620)
     new c639d6f  HBASE-26136 Backport HBASE-25723 "Temporarily remove the trace support for RegionScanner.next" to branch-2 (#3623)
     new dd1bdc8  HBASE-26137 Backport HBASE-25732 "Change the command line argument for tracing after upgrading opentelemtry to 1.0.0" to branch-2 (#3624)
     new a746e19  HBASE-26138 Backport HBASE-25733 "Upgrade opentelemetry to 1.0.1" to branch-2 (#3628)
     new 65b5b9b  HBASE-26139 Backport HBASE-23762 "Add documentation on how to enable and view tracing with OpenTelemetry" to branch-2 (#3629)
     new ab431fc  HBASE-26140 Backport HBASE-25778 "The tracinig implementation for AsyncConnectionImpl.getHbck is incorrect" to branch-2 (#3631)
     new 625093f  HBASE-26168 Backport HBASE-25811 "The client integration test is failing after HBASE-22120 merged" into branch-2 (#3634)

The 18 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 bin/hbase                                          |  10 +
 conf/hbase-env.cmd                                 |   5 +
 conf/hbase-env.sh                                  |   5 +
 hbase-assembly/pom.xml                             |   6 +
 hbase-assembly/src/main/assembly/client.xml        |   8 +
 .../src/main/assembly/hadoop-three-compat.xml      |   8 +
 .../hadoop/hbase/io/asyncfs/AsyncFSTestBase.java   |   3 -
 hbase-client/pom.xml                               |  14 +-
 .../client/AbstractRpcBasedConnectionRegistry.java |  52 ++-
 .../hadoop/hbase/client/AsyncConnection.java       |   4 +-
 .../hadoop/hbase/client/AsyncConnectionImpl.java   | 109 +++---
 .../hadoop/hbase/client/AsyncRegionLocator.java    | 165 +++++---
 .../hbase/client/AsyncRequestFutureImpl.java       |  14 +-
 .../org/apache/hadoop/hbase/client/AsyncTable.java |  31 +-
 .../hbase/client/AsyncTableRegionLocatorImpl.java  |  18 +-
 .../hadoop/hbase/client/ConnectionFactory.java     |  53 +--
 .../hadoop/hbase/client/RawAsyncTableImpl.java     | 345 +++++++++--------
 .../client/ResultBoundedCompletionService.java     |   4 +-
 .../hadoop/hbase/client/ZKConnectionRegistry.java  |  36 +-
 .../apache/hadoop/hbase/ipc/AbstractRpcClient.java |  81 ++--
 .../hadoop/hbase/ipc/BlockingRpcConnection.java    |  18 +-
 .../java/org/apache/hadoop/hbase/ipc/Call.java     |   9 +-
 .../java/org/apache/hadoop/hbase/ipc/IPCUtil.java  |  12 +-
 .../hadoop/hbase/ipc/NettyRpcDuplexHandler.java    |   8 +-
 .../hbase/client/TestAsyncConnectionTracing.java   | 112 ++++++
 .../client/TestAsyncRegionLocatorTracing.java      | 157 ++++++++
 .../hadoop/hbase/client/TestAsyncTableTracing.java | 417 +++++++++++++++++++++
 hbase-common/pom.xml                               |   8 +-
 .../hbase/trace/HBaseHTraceConfiguration.java      |  80 ----
 .../hadoop/hbase/trace/SpanReceiverHost.java       | 120 ------
 .../org/apache/hadoop/hbase/trace/TraceUtil.java   | 231 ++++++++----
 hbase-external-blockcache/pom.xml                  |   4 -
 .../hadoop/hbase/io/hfile/MemcachedBlockCache.java |  14 +-
 hbase-it/pom.xml                                   |   4 +-
 .../IntegrationTestTableMapReduceUtil.java         |   1 -
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java     |  22 +-
 .../trace/IntegrationTestSendTraceRequests.java    | 135 +++----
 hbase-mapreduce/pom.xml                            |   4 +-
 .../hadoop/hbase/mapreduce/TableMapReduceUtil.java |   5 +-
 .../apache/hadoop/hbase/PerformanceEvaluation.java |  29 +-
 hbase-protocol-shaded/pom.xml                      |   4 -
 .../src/main/protobuf/Tracing.proto                |  14 +-
 hbase-server/pom.xml                               |  14 +-
 .../apache/hadoop/hbase/executor/EventHandler.java |  16 +-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java     |  11 +-
 .../org/apache/hadoop/hbase/ipc/CallRunner.java    |  47 +--
 .../org/apache/hadoop/hbase/ipc/ServerCall.java    |  14 +
 .../hadoop/hbase/ipc/ServerRpcConnection.java      | 221 ++++++-----
 .../org/apache/hadoop/hbase/master/HMaster.java    |   2 -
 .../hadoop/hbase/master/HMasterCommandLine.java    |   6 +-
 .../apache/hadoop/hbase/regionserver/HRegion.java  | 184 +++++----
 .../hadoop/hbase/regionserver/HRegionServer.java   |   9 -
 .../regionserver/HRegionServerCommandLine.java     |   8 +-
 .../hadoop/hbase/regionserver/MemStoreFlusher.java |  14 +-
 .../hbase/regionserver/RegionScannerImpl.java      |  43 ++-
 .../hbase/regionserver/wal/AbstractFSWAL.java      |  70 +++-
 .../hadoop/hbase/regionserver/wal/AsyncFSWAL.java  | 104 ++---
 .../hadoop/hbase/regionserver/wal/FSHLog.java      |  42 +--
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |  13 +-
 .../hadoop/hbase/executor/TestExecutorService.java |   4 +-
 .../apache/hadoop/hbase/ipc/AbstractTestIPC.java   |  87 ++++-
 .../org/apache/hadoop/hbase/ipc/TestNettyIPC.java  |   1 +
 .../hbase/regionserver/TestAtomicOperation.java    |   2 +-
 .../hadoop/hbase/regionserver/TestHRegion.java     |   4 +-
 .../hbase/regionserver/TestHRegionTracing.java     | 183 +++++++++
 .../regionserver/wal/TestAsyncFSWALDurability.java |   8 +-
 .../regionserver/wal/TestFSHLogDurability.java     |   8 +-
 .../apache/hadoop/hbase/trace/TestHTraceHooks.java | 134 -------
 .../org/apache/hadoop/hbase/trace/TraceTree.java   | 148 --------
 .../org/apache/hadoop/hbase/wal/FaultyFSLog.java   |  10 +-
 .../hadoop/hbase/wal/WALPerformanceEvaluation.java |  73 ++--
 hbase-shaded/hbase-shaded-client/pom.xml           |   1 +
 hbase-shaded/hbase-shaded-testing-util/pom.xml     |   1 +
 hbase-shaded/pom.xml                               |   1 +
 hbase-shell/src/main/ruby/shell/commands/trace.rb  |  43 +--
 hbase-zookeeper/pom.xml                            |   4 +
 .../hbase/zookeeper/RecoverableZooKeeper.java      |  77 +++-
 pom.xml                                            |  44 ++-
 src/main/asciidoc/_chapters/tracing.adoc           |  57 +++
 src/main/asciidoc/_chapters/upgrading.adoc         |   4 +-
 src/main/asciidoc/book.adoc                        |   1 +
 81 files changed, 2495 insertions(+), 1597 deletions(-)
 create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncConnectionTracing.java
 create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java
 create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java
 delete mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/trace/HBaseHTraceConfiguration.java
 delete mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/trace/SpanReceiverHost.java
 create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
 delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java
 delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TraceTree.java
 create mode 100644 src/main/asciidoc/_chapters/tracing.adoc

[hbase] 09/18: HBASE-26132 Backport HBASE-25535 "Set span kind to CLIENT in AbstractRpcClient" to branch-2 (#3607)

Posted by ta...@apache.org.
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 8b7450f5d8480e0243978b279eaaba5fb1c4994b
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Thu Aug 19 20:12:37 2021 -0700

    HBASE-26132 Backport HBASE-25535 "Set span kind to CLIENT in AbstractRpcClient" to branch-2 (#3607)
    
    9/17 commits of HBASE-22120, original commit bb8c4967f8ce2c89ebaf1ddc5d8a1bf55f1e20d3
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../apache/hadoop/hbase/ipc/AbstractRpcClient.java |  2 +-
 .../org/apache/hadoop/hbase/trace/TraceUtil.java   |  7 +++++++
 .../apache/hadoop/hbase/ipc/AbstractTestIPC.java   | 23 +++++++++++++++++-----
 3 files changed, 26 insertions(+), 6 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index fa7dfb1..bb7f4af 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -395,7 +395,7 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
   private Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
     final Message param, Message returnType, final User ticket, final Address addr,
     final RpcCallback<Message> callback) {
-    Span span = TraceUtil.createSpan("RpcClient.callMethod")
+    Span span = TraceUtil.createClientSpan("RpcClient.callMethod")
       .setAttribute(TraceUtil.RPC_SERVICE_KEY, md.getService().getName())
       .setAttribute(TraceUtil.RPC_METHOD_KEY, md.getName())
       .setAttribute(TraceUtil.REMOTE_HOST_KEY, addr.getHostName())
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
index 886a4a9..2a6b6b6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
@@ -106,6 +106,13 @@ public final class TraceUtil {
   }
 
   /**
+   * Create a span with {@link Kind#CLIENT}.
+   */
+  public static Span createClientSpan(String name) {
+    return createSpan(name, Kind.CLIENT);
+  }
+
+  /**
    * Trace an asynchronous operation for a table.
    */
   public static <T> CompletableFuture<T> tracedFuture(Supplier<CompletableFuture<T>> action,
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index ffecb09..c4951b2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -34,6 +34,7 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 import static org.mockito.internal.verification.VerificationModeFactory.times;
 
+import io.opentelemetry.api.trace.Span.Kind;
 import io.opentelemetry.api.trace.StatusCode;
 import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule;
 import io.opentelemetry.sdk.trace.data.SpanData;
@@ -455,7 +456,8 @@ public abstract class AbstractTestIPC {
     return traceRule.getSpans().stream().filter(s -> s.getName().equals(name)).findFirst().get();
   }
 
-  private void assertRpcAttribute(SpanData data, String methodName, InetSocketAddress addr) {
+  private void assertRpcAttribute(SpanData data, String methodName, InetSocketAddress addr,
+    Kind kind) {
     assertEquals(SERVICE.getDescriptorForType().getName(),
       data.getAttributes().get(TraceUtil.RPC_SERVICE_KEY));
     assertEquals(methodName, data.getAttributes().get(TraceUtil.RPC_METHOD_KEY));
@@ -463,6 +465,13 @@ public abstract class AbstractTestIPC {
       assertEquals(addr.getHostName(), data.getAttributes().get(TraceUtil.REMOTE_HOST_KEY));
       assertEquals(addr.getPort(), data.getAttributes().get(TraceUtil.REMOTE_PORT_KEY).intValue());
     }
+    assertEquals(kind, data.getKind());
+  }
+
+  private void assertRemoteSpan() {
+    SpanData data = waitSpan("RpcServer.process");
+    assertTrue(data.getParentSpanContext().isRemote());
+    assertEquals(Kind.SERVER, data.getKind());
   }
 
   @Test
@@ -474,8 +483,10 @@ public abstract class AbstractTestIPC {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       stub.pause(null, PauseRequestProto.newBuilder().setMs(100).build());
-      assertRpcAttribute(waitSpan("RpcClient.callMethod"), "pause", rpcServer.getListenerAddress());
-      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "pause", null);
+      assertRpcAttribute(waitSpan("RpcClient.callMethod"), "pause", rpcServer.getListenerAddress(),
+        Kind.CLIENT);
+      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "pause", null, Kind.INTERNAL);
+      assertRemoteSpan();
       assertSameTraceId();
       for (SpanData data : traceRule.getSpans()) {
         assertThat(
@@ -487,8 +498,10 @@ public abstract class AbstractTestIPC {
       traceRule.clearSpans();
       assertThrows(ServiceException.class,
         () -> stub.error(null, EmptyRequestProto.getDefaultInstance()));
-      assertRpcAttribute(waitSpan("RpcClient.callMethod"), "error", rpcServer.getListenerAddress());
-      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "error", null);
+      assertRpcAttribute(waitSpan("RpcClient.callMethod"), "error", rpcServer.getListenerAddress(),
+        Kind.CLIENT);
+      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "error", null, Kind.INTERNAL);
+      assertRemoteSpan();
       assertSameTraceId();
       for (SpanData data : traceRule.getSpans()) {
         assertEquals(StatusCode.ERROR, data.getStatus().getStatusCode());

[hbase] 13/18: HBASE-26136 Backport HBASE-25723 "Temporarily remove the trace support for RegionScanner.next" to branch-2 (#3623)

Posted by ta...@apache.org.
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 c639d6f353b09c1466e2eb7ec76e4d4e0e42009d
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Tue Aug 24 17:45:04 2021 -0700

    HBASE-26136 Backport HBASE-25723 "Temporarily remove the trace support for RegionScanner.next" to branch-2 (#3623)
    
    13/17 commits of HBASE-22120, original commit 7f90c2201f6a17d2e2d031505c35ae7c2b1ed7ea
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../hbase/regionserver/RegionScannerImpl.java      | 58 +++++++++++-----------
 .../hbase/regionserver/TestHRegionTracing.java     |  1 -
 2 files changed, 28 insertions(+), 31 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java
index b2e793f..881f423 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java
@@ -255,39 +255,37 @@ class RegionScannerImpl implements RegionScanner, Shipper, RpcCallback {
 
   @Override
   public boolean nextRaw(List<Cell> outResults, ScannerContext scannerContext) throws IOException {
-    return TraceUtil.trace(() -> {
-      if (storeHeap == null) {
-        // scanner is closed
-        throw new UnknownScannerException("Scanner was closed");
-      }
-      boolean moreValues = false;
-      if (outResults.isEmpty()) {
-        // Usually outResults is empty. This is true when next is called
-        // to handle scan or get operation.
-        moreValues = nextInternal(outResults, scannerContext);
-      } else {
-        List<Cell> tmpList = new ArrayList<>();
-        moreValues = nextInternal(tmpList, scannerContext);
-        outResults.addAll(tmpList);
-      }
+    if (storeHeap == null) {
+      // scanner is closed
+      throw new UnknownScannerException("Scanner was closed");
+    }
+    boolean moreValues = false;
+    if (outResults.isEmpty()) {
+      // Usually outResults is empty. This is true when next is called
+      // to handle scan or get operation.
+      moreValues = nextInternal(outResults, scannerContext);
+    } else {
+      List<Cell> tmpList = new ArrayList<>();
+      moreValues = nextInternal(tmpList, scannerContext);
+      outResults.addAll(tmpList);
+    }
 
-      region.addReadRequestsCount(1);
-      if (region.getMetrics() != null) {
-        region.getMetrics().updateReadRequestCount();
-      }
+    region.addReadRequestsCount(1);
+    if (region.getMetrics() != null) {
+      region.getMetrics().updateReadRequestCount();
+    }
 
-      // If the size limit was reached it means a partial Result is being returned. Returning a
-      // partial Result means that we should not reset the filters; filters should only be reset in
-      // between rows
-      if (!scannerContext.mayHaveMoreCellsInRow()) {
-        resetFilters();
-      }
+    // If the size limit was reached it means a partial Result is being returned. Returning a
+    // partial Result means that we should not reset the filters; filters should only be reset in
+    // between rows
+    if (!scannerContext.mayHaveMoreCellsInRow()) {
+      resetFilters();
+    }
 
-      if (isFilterDoneInternal()) {
-        moreValues = false;
-      }
-      return moreValues;
-    }, () -> region.createRegionSpan("RegionScanner.next"));
+    if (isFilterDoneInternal()) {
+      moreValues = false;
+    }
+    return moreValues;
   }
 
   /**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
index 17c3eb4..e5779b4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
@@ -178,7 +178,6 @@ public class TestHRegionTracing {
     }
     assertSpan("Region.getScanner");
     assertSpan("RegionScanner.reseek");
-    assertSpan("RegionScanner.next");
     assertSpan("RegionScanner.close");
   }
 }

[hbase] 14/18: HBASE-26137 Backport HBASE-25732 "Change the command line argument for tracing after upgrading opentelemtry to 1.0.0" to branch-2 (#3624)

Posted by ta...@apache.org.
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 dd1bdc8aff73ec1d01db83ce45e8506ef926bb91
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Wed Aug 25 09:01:19 2021 -0700

    HBASE-26137 Backport HBASE-25732 "Change the command line argument for tracing after upgrading opentelemtry to 1.0.0" to branch-2 (#3624)
    
    14/17 commits of HBASE-22120, original commit 8df9bebdd367d52a32b08c18a7cf4f9c2d712071
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 conf/hbase-env.cmd | 5 +++++
 conf/hbase-env.sh  | 6 +++---
 2 files changed, 8 insertions(+), 3 deletions(-)

diff --git a/conf/hbase-env.cmd b/conf/hbase-env.cmd
index 1f1c3e3..4beebf6 100644
--- a/conf/hbase-env.cmd
+++ b/conf/hbase-env.cmd
@@ -81,3 +81,8 @@ set HBASE_OPTS=%HBASE_OPTS% "-XX:+UseConcMarkSweepGC" "-Djava.net.preferIPv4Stac
 
 @rem Tell HBase whether it should manage it's own instance of ZooKeeper or not.
 @rem set HBASE_MANAGES_ZK=true
+
+@rem Uncomment to enable trace, you can change the options to use other exporters such as jaeger or
+@rem zipkin. See https://github.com/open-telemetry/opentelemetry-java-instrumentation on how to
+@rem configure exporters and other components through system properties.
+@rem set HBASE_TRACE_OPTS="-Dotel.resource.attributes=service.name=HBase -Dotel.traces.exporter=logging otel.metrics.exporter=none"
diff --git a/conf/hbase-env.sh b/conf/hbase-env.sh
index b3c4baf..d1d9a6f 100644
--- a/conf/hbase-env.sh
+++ b/conf/hbase-env.sh
@@ -146,6 +146,6 @@
 # export HBASE_JSHELL_ARGS="--startup DEFAULT --startup PRINTING --startup hbase_startup.jsh"
 
 # Uncomment to enable trace, you can change the options to use other exporters such as jaeger or
-# zipkin. See https://github.com/open-telemetry/opentelemetry-java-instrumentation on how to config
-# exporters and other components through system properties.
-# export HBASE_TRACE_OPTS="-Dotel.config.sampler.probability=0.1 -Dotel.exporter=logging"
+# zipkin. See https://github.com/open-telemetry/opentelemetry-java-instrumentation on how to
+# configure exporters and other components through system properties.
+# export HBASE_TRACE_OPTS="-Dotel.resource.attributes=service.name=HBase -Dotel.traces.exporter=logging otel.metrics.exporter=none"

[hbase] 06/18: HBASE-26129 Backport HBASE-25481 "Add host and port attribute when tracing rpc call at client side" to branch-2 (#3589)

Posted by ta...@apache.org.
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 83a27389a025f4d6b9481f4b66a8e6799c84f8b0
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Tue Aug 17 09:24:10 2021 -0700

    HBASE-26129 Backport HBASE-25481 "Add host and port attribute when tracing rpc call at client side" to branch-2 (#3589)
    
    6/17 commits of HBASE-22120, original commit ae2c62ffaad5ba4c976b0a79c10a365edf2844fd
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by:  Peter Somogyi  <ps...@apache.org>
---
 .../org/apache/hadoop/hbase/ipc/AbstractRpcClient.java  |  4 +++-
 .../java/org/apache/hadoop/hbase/trace/TraceUtil.java   |  4 ++++
 .../org/apache/hadoop/hbase/ipc/AbstractTestIPC.java    | 17 ++++++++++-------
 3 files changed, 17 insertions(+), 8 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index b671095..878f9cf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -397,7 +397,9 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
     final RpcCallback<Message> callback) {
     Span span = TraceUtil.createSpan("RpcClient.callMethod")
       .setAttribute(TraceUtil.RPC_SERVICE_KEY, md.getService().getName())
-      .setAttribute(TraceUtil.RPC_METHOD_KEY, md.getName());
+      .setAttribute(TraceUtil.RPC_METHOD_KEY, md.getName())
+      .setAttribute(TraceUtil.REMOTE_HOST_KEY, addr.getHostName())
+      .setAttribute(TraceUtil.REMOTE_PORT_KEY, addr.getPort());
     try (Scope scope = span.makeCurrent()) {
       final MetricsConnection.CallStats cs = MetricsConnection.newCallStats();
       cs.setStartTime(EnvironmentEdgeManager.currentTime());
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
index d0da071..8eb2399 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
@@ -54,6 +54,10 @@ public final class TraceUtil {
   public static final AttributeKey<String> SERVER_NAME_KEY =
     AttributeKey.stringKey("db.hbase.server.name");
 
+  public static final AttributeKey<String> REMOTE_HOST_KEY = SemanticAttributes.NET_PEER_NAME;
+
+  public static final AttributeKey<Long> REMOTE_PORT_KEY = SemanticAttributes.NET_PEER_PORT;
+
   private TraceUtil() {
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index 4aca764..ffecb09 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -455,11 +455,14 @@ public abstract class AbstractTestIPC {
     return traceRule.getSpans().stream().filter(s -> s.getName().equals(name)).findFirst().get();
   }
 
-  private void assertRpcAttribute(SpanData data, String methodName) {
+  private void assertRpcAttribute(SpanData data, String methodName, InetSocketAddress addr) {
     assertEquals(SERVICE.getDescriptorForType().getName(),
       data.getAttributes().get(TraceUtil.RPC_SERVICE_KEY));
-    assertEquals(methodName,
-      data.getAttributes().get(TraceUtil.RPC_METHOD_KEY));
+    assertEquals(methodName, data.getAttributes().get(TraceUtil.RPC_METHOD_KEY));
+    if (addr != null) {
+      assertEquals(addr.getHostName(), data.getAttributes().get(TraceUtil.REMOTE_HOST_KEY));
+      assertEquals(addr.getPort(), data.getAttributes().get(TraceUtil.REMOTE_PORT_KEY).intValue());
+    }
   }
 
   @Test
@@ -471,8 +474,8 @@ public abstract class AbstractTestIPC {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       stub.pause(null, PauseRequestProto.newBuilder().setMs(100).build());
-      assertRpcAttribute(waitSpan("RpcClient.callMethod"), "pause");
-      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "pause");
+      assertRpcAttribute(waitSpan("RpcClient.callMethod"), "pause", rpcServer.getListenerAddress());
+      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "pause", null);
       assertSameTraceId();
       for (SpanData data : traceRule.getSpans()) {
         assertThat(
@@ -484,8 +487,8 @@ public abstract class AbstractTestIPC {
       traceRule.clearSpans();
       assertThrows(ServiceException.class,
         () -> stub.error(null, EmptyRequestProto.getDefaultInstance()));
-      assertRpcAttribute(waitSpan("RpcClient.callMethod"), "error");
-      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "error");
+      assertRpcAttribute(waitSpan("RpcClient.callMethod"), "error", rpcServer.getListenerAddress());
+      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "error", null);
       assertSameTraceId();
       for (SpanData data : traceRule.getSpans()) {
         assertEquals(StatusCode.ERROR, data.getStatus().getStatusCode());

[hbase] 16/18: HBASE-26139 Backport HBASE-23762 "Add documentation on how to enable and view tracing with OpenTelemetry" to branch-2 (#3629)

Posted by ta...@apache.org.
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 65b5b9b2a867957232719005740bf30ac988accb
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Thu Aug 26 09:36:05 2021 -0700

    HBASE-26139 Backport HBASE-23762 "Add documentation on how to enable and view tracing with OpenTelemetry" to branch-2 (#3629)
    
    16/17 commits of HBASE-22120, original commit  be4503d9f82f044fbfce21c8a42d0b1684607238
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 src/main/asciidoc/_chapters/tracing.adoc   | 57 ++++++++++++++++++++++++++++++
 src/main/asciidoc/_chapters/upgrading.adoc |  4 +--
 src/main/asciidoc/book.adoc                |  1 +
 3 files changed, 60 insertions(+), 2 deletions(-)

diff --git a/src/main/asciidoc/_chapters/tracing.adoc b/src/main/asciidoc/_chapters/tracing.adoc
new file mode 100644
index 0000000..cc9f093
--- /dev/null
+++ b/src/main/asciidoc/_chapters/tracing.adoc
@@ -0,0 +1,57 @@
+////
+/**
+ *
+ * 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.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+////
+
+[[tracing]]
+= Tracing
+:doctype: book
+:numbered:
+:toc: left
+:icons: font
+:experimental:
+
+== Overview
+
+HBase used to depend on the HTrace project for tracing. After the Apache HTrace project moved to the Attic/retired, we decided to move to https://opentelemetry.io[OpenTelemetry] in https://issues.apache.org/jira/browse/HBASE-22120[HBASE-22120].
+
+The basic support for tracing has been done, where we added tracing for async client, rpc, region read/write/scan operation, and WAL. We use opentelemetry-api to implement the tracing support manually by code, as our code base is way too complicated to be instrumented through a java agent. But notice that you still need to attach the opentelemetry java agent to enable tracing. Please see the official site for https://opentelemetry.io/[OpenTelemetry] and the documentation for https://gith [...]
+
+== Usage
+
+=== Enable Tracing
+
+See this section in hbase-env.sh
+
+[source,shell]
+----
+# Uncomment to enable trace, you can change the options to use other exporters such as jaeger or
+# zipkin. See https://github.com/open-telemetry/opentelemetry-java-instrumentation on how to
+# configure exporters and other components through system properties.
+# export HBASE_TRACE_OPTS="-Dotel.resource.attributes=service.name=HBase -Dotel.traces.exporter=logging otel.metrics.exporter=none"
+----
+
+Uncomment this line to enable tracing. The default config is to output the tracing data to log. Please see the documentation for https://github.com/open-telemetry/opentelemetry-java-instrumentation[opentelemetry-java-instrumentation] for more details on how to export tracing data to other tracing system such as OTel collector, jaeger or zipkin, what does the _service.name_ mean, and how to change the sampling rate, etc.
+
+NOTE: The https://github.com/open-telemetry/opentelemetry-java/blob/v1.0.1/exporters/logging/src/main/java/io/opentelemetry/exporter/logging/LoggingSpanExporter.java[LoggingSpanExporter] uses java.util.logging(jul) for logging tracing data, and the logger is initialized in opentelemetry java agent, which seems to be ahead of our jul to slf4j bridge initialization, so it will always log the tracing data to console. We highly suggest that you use other tracing systems to collect and view t [...]
+
+=== Performance Impact
+
+According to the result in https://issues.apache.org/jira/browse/HBASE-25658[HBASE-25658], the performance impact is minimal. Of course the test cluster is not under heavy load, so if you find out that enabling tracing would impact the performance, try to lower the sampling rate. See documentation for configuring https://github.com/open-telemetry/opentelemetry-java/blob/main/sdk-extensions/autoconfigure/README.md#sampler[sampler] for more details.
+
diff --git a/src/main/asciidoc/_chapters/upgrading.adoc b/src/main/asciidoc/_chapters/upgrading.adoc
index cf1568e..d4f9e37 100644
--- a/src/main/asciidoc/_chapters/upgrading.adoc
+++ b/src/main/asciidoc/_chapters/upgrading.adoc
@@ -634,11 +634,11 @@ The Java client API for HBase has a number of changes that break both source and
 .Tracing implementation changes
 The backing implementation of HBase's tracing features was updated from Apache HTrace 3 to HTrace 4, which includes several breaking changes. While HTrace 3 and 4 can coexist in the same runtime, they will not integrate with each other, leading to disjoint trace information.
 
-The internal changes to HBase during this upgrade were sufficient for compilation, but it has not been confirmed that there are no regressions in tracing functionality. Please consider this feature expiremental for the immediate future.
+The internal changes to HBase during this upgrade were sufficient for compilation, but it has not been confirmed that there are no regressions in tracing functionality. Please consider this feature experimental for the immediate future.
 
 If you previously relied on client side tracing integrated with HBase operations, it is recommended that you upgrade your usage to HTrace 4 as well.
 
-After the Apache HTrace project moved to the Attic/retired, the traces in HBase are left broken and unmaintained since HBase 2.0. A new project link:https://issues.apache.org/jira/browse/HBASE-22120[HBASE-22120] will replace HTrace with OpenTracing.
+After the Apache HTrace project moved to the Attic/retired, the traces in HBase are left broken and unmaintained since HBase 2.0. A new project link:https://issues.apache.org/jira/browse/HBASE-22120[HBASE-22120] will replace HTrace with OpenTelemetry. It will be shipped in 3.0.0 release. Please see the reference guide section <<tracing>> for more details.
 
 [[upgrade2.0.hfile.compatability]]
 .HFile lose forward compatability
diff --git a/src/main/asciidoc/book.adoc b/src/main/asciidoc/book.adoc
index e0df010..c264b06 100644
--- a/src/main/asciidoc/book.adoc
+++ b/src/main/asciidoc/book.adoc
@@ -86,6 +86,7 @@ include::_chapters/amv2.adoc[]
 include::_chapters/zookeeper.adoc[]
 include::_chapters/community.adoc[]
 include::_chapters/hbtop.adoc[]
+include::_chapters/tracing.adoc[]
 
 = Appendix
 

[hbase] 02/18: HBASE-26125 Backport HBASE-25401 "Add trace support for async call in rpc client" to branch-2 (#3543)

Posted by ta...@apache.org.
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 4dfc59c19e7b380483dc422ec1e503f704430c67
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Mon Aug 2 11:33:19 2021 -0700

    HBASE-26125 Backport HBASE-25401 "Add trace support for async call in rpc client" to branch-2 (#3543)
    
    2/17 commits of HBASE-22120
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 .../apache/hadoop/hbase/ipc/AbstractRpcClient.java |  75 ++++---
 .../hadoop/hbase/ipc/BlockingRpcConnection.java    |  21 +-
 .../java/org/apache/hadoop/hbase/ipc/Call.java     |   2 +-
 .../java/org/apache/hadoop/hbase/ipc/IPCUtil.java  |  12 +-
 .../hadoop/hbase/ipc/NettyRpcDuplexHandler.java    |   8 +-
 .../org/apache/hadoop/hbase/trace/TraceUtil.java   |   4 +-
 .../src/main/protobuf/Tracing.proto                |  14 +-
 hbase-server/pom.xml                               |  10 +
 .../org/apache/hadoop/hbase/ipc/CallRunner.java    |  19 +-
 .../org/apache/hadoop/hbase/ipc/ServerCall.java    |  14 ++
 .../hadoop/hbase/ipc/ServerRpcConnection.java      | 222 ++++++++++++---------
 .../apache/hadoop/hbase/ipc/AbstractTestIPC.java   |  59 +++++-
 .../org/apache/hadoop/hbase/ipc/TestNettyIPC.java  |   1 +
 pom.xml                                            |  12 +-
 14 files changed, 299 insertions(+), 174 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 4bbb729..9117fef 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -21,6 +21,9 @@ package org.apache.hadoop.hbase.ipc;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.wrapException;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.net.SocketAddress;
 import java.util.Collection;
@@ -38,6 +41,7 @@ import org.apache.hadoop.hbase.codec.KeyValueCodec;
 import org.apache.hadoop.hbase.net.Address;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.PoolMap;
 import org.apache.hadoop.hbase.util.Threads;
@@ -365,7 +369,7 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
   protected abstract T createConnection(ConnectionId remoteId) throws IOException;
 
   private void onCallFinished(Call call, HBaseRpcController hrc, Address addr,
-      RpcCallback<Message> callback) {
+    RpcCallback<Message> callback) {
     call.callStats.setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.getStartTime());
     if (metrics != null) {
       metrics.updateRpc(call.md, call.param, call.callStats);
@@ -388,44 +392,59 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
     }
   }
 
-  Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
+  private Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
       final Message param, Message returnType, final User ticket,
       final Address addr, final RpcCallback<Message> callback) {
-    final MetricsConnection.CallStats cs = MetricsConnection.newCallStats();
-    cs.setStartTime(EnvironmentEdgeManager.currentTime());
-
-    if (param instanceof ClientProtos.MultiRequest) {
-      ClientProtos.MultiRequest req = (ClientProtos.MultiRequest) param;
-      int numActions = 0;
-      for (ClientProtos.RegionAction regionAction : req.getRegionActionList()) {
-        numActions += regionAction.getActionCount();
-      }
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("RpcClient.callMethod." + md.getFullName())
+      .startSpan();
+    try (Scope scope = span.makeCurrent()) {
+      final MetricsConnection.CallStats cs = MetricsConnection.newCallStats();
+      cs.setStartTime(EnvironmentEdgeManager.currentTime());
+
+      if (param instanceof ClientProtos.MultiRequest) {
+        ClientProtos.MultiRequest req = (ClientProtos.MultiRequest) param;
+        int numActions = 0;
+        for (ClientProtos.RegionAction regionAction : req.getRegionActionList()) {
+          numActions += regionAction.getActionCount();
+        }
 
-      cs.setNumActionsPerServer(numActions);
-    }
+        cs.setNumActionsPerServer(numActions);
+      }
 
-    final AtomicInteger counter = concurrentCounterCache.getUnchecked(addr);
-    Call call = new Call(nextCallId(), md, param, hrc.cellScanner(), returnType,
+      final AtomicInteger counter = concurrentCounterCache.getUnchecked(addr);
+      Call call = new Call(nextCallId(), md, param, hrc.cellScanner(), returnType,
         hrc.getCallTimeout(), hrc.getPriority(), new RpcCallback<Call>() {
           @Override
           public void run(Call call) {
-            counter.decrementAndGet();
-            onCallFinished(call, hrc, addr, callback);
+            try (Scope scope = call.span.makeCurrent()) {
+              counter.decrementAndGet();
+              onCallFinished(call, hrc, addr, callback);
+            } finally {
+              if (hrc.failed()) {
+                span.setStatus(StatusCode.ERROR);
+                span.recordException(hrc.getFailed());
+              } else {
+                span.setStatus(StatusCode.OK);
+              }
+              span.end();
+            }
           }
         }, cs);
-    ConnectionId remoteId = new ConnectionId(ticket, md.getService().getName(), addr);
-    int count = counter.incrementAndGet();
-    try {
-      if (count > maxConcurrentCallsPerServer) {
-        throw new ServerTooBusyException(addr, count);
+      ConnectionId remoteId = new ConnectionId(ticket, md.getService().getName(), addr);
+      int count = counter.incrementAndGet();
+      try {
+        if (count > maxConcurrentCallsPerServer) {
+          throw new ServerTooBusyException(addr, count);
+        }
+        cs.setConcurrentCallsPerServer(count);
+        T connection = getConnection(remoteId);
+        connection.sendRequest(call, hrc);
+      } catch (Exception e) {
+        call.setException(toIOE(e));
+        span.end();
       }
-      cs.setConcurrentCallsPerServer(count);
-      T connection = getConnection(remoteId);
-      connection.sendRequest(call, hrc);
-    } catch (Exception e) {
-      call.setException(toIOE(e));
+      return call;
     }
-    return call;
   }
 
   private static Address createAddr(ServerName sn) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
index 1a5cb73..eb8e1d9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
@@ -24,8 +24,6 @@ import static org.apache.hadoop.hbase.ipc.IPCUtil.isFatalConnectionException;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.write;
 
-import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Context;
 import io.opentelemetry.context.Scope;
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -57,7 +55,6 @@ import org.apache.hadoop.hbase.security.HBaseSaslRpcClient;
 import org.apache.hadoop.hbase.security.SaslUtil;
 import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
 import org.apache.hadoop.hbase.security.provider.SaslClientAuthenticationProvider;
-import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.io.IOUtils;
@@ -192,8 +189,8 @@ class BlockingRpcConnection extends RpcConnection implements Runnable {
           if (call.isDone()) {
             continue;
           }
-          try {
-            tracedWriteRequest(call);
+          try (Scope scope = call.span.makeCurrent()) {
+            writeRequest(call);
           } catch (IOException e) {
             // exception here means the call has not been added to the pendingCalls yet, so we need
             // to fail it by our own.
@@ -594,16 +591,6 @@ class BlockingRpcConnection extends RpcConnection implements Runnable {
     this.out = new DataOutputStream(new BufferedOutputStream(saslRpcClient.getOutputStream()));
   }
 
-  private void tracedWriteRequest(Call call) throws IOException {
-    Span span = TraceUtil.getGlobalTracer().spanBuilder("RpcClientImpl.tracedWriteRequest")
-      .setParent(Context.current().with(call.span)).startSpan();
-    try (Scope scope = span.makeCurrent()) {
-      writeRequest(call);
-    } finally {
-      span.end();
-    }
-  }
-
   /**
    * Initiates a call by sending the parameter to the remote server. Note: this is not called from
    * the Connection thread, but by other threads.
@@ -811,7 +798,9 @@ class BlockingRpcConnection extends RpcConnection implements Runnable {
         if (callSender != null) {
           callSender.sendCall(call);
         } else {
-          tracedWriteRequest(call);
+          // this is in the same thread with the caller so do not need to attach the trace context
+          // again.
+          writeRequest(call);
         }
       }
     });
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
index 113f731..8d23d92 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
@@ -61,7 +61,7 @@ class Call {
   final Span span;
   Timeout timeoutTask;
 
-  protected Call(int id, final Descriptors.MethodDescriptor md, Message param,
+  Call(int id, final Descriptors.MethodDescriptor md, Message param,
       final CellScanner cells, final Message responseDefaultType, int timeout, int priority,
       RpcCallback<Call> callback, MetricsConnection.CallStats callStats) {
     this.param = param;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
index 42ad33a..fd42214 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import io.opentelemetry.api.GlobalOpenTelemetry;
+import io.opentelemetry.context.Context;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.lang.reflect.InvocationTargetException;
@@ -49,6 +51,7 @@ import org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocal;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo;
 
 /**
  * Utility to help ipc'ing.
@@ -112,11 +115,10 @@ class IPCUtil {
   static RequestHeader buildRequestHeader(Call call, CellBlockMeta cellBlockMeta) {
     RequestHeader.Builder builder = RequestHeader.newBuilder();
     builder.setCallId(call.id);
-    //TODO handle htrace API change, see HBASE-18895
-    /*if (call.span != null) {
-      builder.setTraceInfo(RPCTInfo.newBuilder().setParentId(call.span.getSpanId())
-          .setTraceId(call.span.getTracerId()));
-    }*/
+    RPCTInfo.Builder traceBuilder = RPCTInfo.newBuilder();
+    GlobalOpenTelemetry.getPropagators().getTextMapPropagator().inject(Context.current(),
+      traceBuilder, (carrier, key, value) -> carrier.putHeaders(key, value));
+    builder.setTraceInfo(traceBuilder.build());
     builder.setMethodName(call.md.getName());
     builder.setRequestParam(call.param != null);
     if (cellBlockMeta != null) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
index f31e3d2..c67d96f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -114,9 +115,12 @@ class NettyRpcDuplexHandler extends ChannelDuplexHandler {
 
   @Override
   public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
-      throws Exception {
+    throws Exception {
     if (msg instanceof Call) {
-      writeRequest(ctx, (Call) msg, promise);
+      Call call = (Call) msg;
+      try (Scope scope = call.span.makeCurrent()) {
+        writeRequest(ctx, call, promise);
+      }
     } else {
       ctx.write(msg, promise);
     }
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
index f7a111f..768de9c 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hbase.trace;
 
-import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.GlobalOpenTelemetry;
 import io.opentelemetry.api.trace.Tracer;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -30,6 +30,6 @@ public final class TraceUtil {
   }
 
   public static Tracer getGlobalTracer() {
-    return OpenTelemetry.getGlobalTracer(INSTRUMENTATION_NAME);
+    return GlobalOpenTelemetry.getTracer(INSTRUMENTATION_NAME);
   }
 }
diff --git a/hbase-protocol-shaded/src/main/protobuf/Tracing.proto b/hbase-protocol-shaded/src/main/protobuf/Tracing.proto
index 64ead84..276a0a7 100644
--- a/hbase-protocol-shaded/src/main/protobuf/Tracing.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/Tracing.proto
@@ -23,12 +23,12 @@ option java_outer_classname = "TracingProtos";
 option java_generate_equals_and_hash = true;
 option optimize_for = SPEED;
 
-//Used to pass through the information necessary to continue
-//a trace after an RPC is made. All we need is the traceid 
-//(so we know the overarching trace this message is a part of), and
-//the id of the current span when this message was sent, so we know 
-//what span caused the new span we will create when this message is received.
+// OpenTelemetry propagates trace context through https://www.w3.org/TR/trace-context/, which
+// is a text-based approach that passes properties with http headers. Here we will also use this
+// approach so we just need a map to store the key value pair.
+
 message RPCTInfo {
-  optional int64 trace_id = 1;
-  optional int64 parent_id = 2;
+  optional int64 trace_id = 1 [deprecated = true];
+  optional int64 parent_id = 2 [deprecated = true];
+  map<string, string> headers = 3;
 }
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index a9a7b72..ef2fad3 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -441,6 +441,16 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk-testing</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.hamcrest</groupId>
       <artifactId>hamcrest-library</artifactId>
       <scope>test</scope>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 3ae089e..203f079 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hbase.ipc;
 
 import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.context.Context;
 import io.opentelemetry.context.Scope;
 import java.net.InetSocketAddress;
 import java.nio.channels.ClosedChannelException;
@@ -73,15 +75,6 @@ public class CallRunner {
     return call;
   }
 
-  /**
-   * Keep for backward compatibility.
-   * @deprecated As of release 2.0, this will be removed in HBase 3.0
-   */
-  @Deprecated
-  public ServerCall<?> getCall() {
-    return (ServerCall<?>) call;
-  }
-
   public void setStatus(MonitoredRPCHandler status) {
     this.status = status;
   }
@@ -130,7 +123,8 @@ public class CallRunner {
       String serviceName = getServiceName();
       String methodName = getMethodName();
       String traceString = serviceName + "." + methodName;
-      Span span = TraceUtil.getGlobalTracer().spanBuilder(traceString).startSpan();
+      Span span = TraceUtil.getGlobalTracer().spanBuilder(traceString)
+        .setParent(Context.current().with(((ServerCall<?>) call).getSpan())).startSpan();
       try (Scope traceScope = span.makeCurrent()) {
         if (!this.rpcServer.isStarted()) {
           InetSocketAddress address = rpcServer.getListenerAddress();
@@ -141,8 +135,12 @@ public class CallRunner {
         resultPair = this.rpcServer.call(call, this.status);
       } catch (TimeoutIOException e){
         RpcServer.LOG.warn("Can not complete this request in time, drop it: " + call);
+        span.recordException(e);
+        span.setStatus(StatusCode.ERROR);
         return;
       } catch (Throwable e) {
+        span.recordException(e);
+        span.setStatus(StatusCode.ERROR);
         if (e instanceof ServerNotRunningYetException) {
           // If ServerNotRunningYetException, don't spew stack trace.
           if (RpcServer.LOG.isTraceEnabled()) {
@@ -161,6 +159,7 @@ public class CallRunner {
         RpcServer.CurCall.set(null);
         if (resultPair != null) {
           this.rpcServer.addCallSize(call.getSize() * -1);
+          span.setStatus(StatusCode.OK);
           sucessful = true;
         }
         span.end();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java
index ff4a521..4a021ce 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.StatusCode;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
@@ -102,6 +104,8 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
   // from WAL side on release
   private final AtomicInteger reference = new AtomicInteger(0x80000000);
 
+  private final Span span;
+
   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NULL_ON_SOME_PATH",
       justification = "Can't figure why this complaint is happening... see below")
   ServerCall(int id, BlockingService service, MethodDescriptor md, RequestHeader header,
@@ -132,6 +136,7 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
     this.bbAllocator = byteBuffAllocator;
     this.cellBlockBuilder = cellBlockBuilder;
     this.reqCleanup = reqCleanup;
+    this.span = Span.current();
   }
 
   /**
@@ -150,6 +155,7 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
     // If the call was run successfuly, we might have already returned the BB
     // back to pool. No worries..Then inputCellBlock will be null
     cleanup();
+    span.end();
   }
 
   @Override
@@ -226,6 +232,10 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
     }
     if (t != null) {
       this.isError = true;
+      span.recordException(t);
+      span.setStatus(StatusCode.ERROR);
+    } else {
+      span.setStatus(StatusCode.OK);
     }
     BufferChain bc = null;
     try {
@@ -560,4 +570,8 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
       return response;
     }
   }
+
+  public Span getSpan() {
+    return span;
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
index 29ce30b..db7f052 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
@@ -19,6 +19,11 @@ package org.apache.hadoop.hbase.ipc;
 
 import static org.apache.hadoop.hbase.HConstants.RPC_HEADER;
 
+import io.opentelemetry.api.GlobalOpenTelemetry;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Context;
+import io.opentelemetry.context.Scope;
+import io.opentelemetry.context.propagation.TextMapPropagator;
 import java.io.ByteArrayInputStream;
 import java.io.Closeable;
 import java.io.DataOutputStream;
@@ -31,13 +36,11 @@ import java.nio.channels.ReadableByteChannel;
 import java.security.GeneralSecurityException;
 import java.util.Objects;
 import java.util.Properties;
-
 import org.apache.commons.crypto.cipher.CryptoCipherFactory;
 import org.apache.commons.crypto.random.CryptoRandom;
 import org.apache.commons.crypto.random.CryptoRandomFactory;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
@@ -53,6 +56,20 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.provider.SaslServerAuthenticationProvider;
 import org.apache.hadoop.hbase.security.provider.SaslServerAuthenticationProviders;
 import org.apache.hadoop.hbase.security.provider.SimpleSaslServerAuthenticationProvider;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.apache.hadoop.security.authorize.ProxyUsers;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteInput;
 import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
@@ -61,6 +78,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescrip
 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.VersionInfo;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
@@ -68,17 +86,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHea
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader;
 import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-import org.apache.hadoop.security.authorize.AuthorizationException;
-import org.apache.hadoop.security.authorize.ProxyUsers;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.TracingProtos.RPCTInfo;
 
 /** Reads calls from a connection and queues them for handling. */
 @edu.umd.cs.findbugs.annotations.SuppressWarnings(
@@ -607,99 +615,115 @@ abstract class ServerRpcConnection implements Closeable {
     ProtobufUtil.mergeFrom(builder, cis, headerSize);
     RequestHeader header = (RequestHeader) builder.build();
     offset += headerSize;
-    int id = header.getCallId();
-    if (RpcServer.LOG.isTraceEnabled()) {
-      RpcServer.LOG.trace("RequestHeader " + TextFormat.shortDebugString(header)
-          + " totalRequestSize: " + totalRequestSize + " bytes");
-    }
-    // Enforcing the call queue size, this triggers a retry in the client
-    // This is a bit late to be doing this check - we have already read in the
-    // total request.
-    if ((totalRequestSize +
-        this.rpcServer.callQueueSizeInBytes.sum()) > this.rpcServer.maxQueueSizeInBytes) {
-      final ServerCall<?> callTooBig = createCall(id, this.service, null, null, null, null,
-        totalRequestSize, null, 0, this.callCleanup);
-      this.rpcServer.metrics.exception(RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION);
-      callTooBig.setResponse(null, null,  RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION,
-        "Call queue is full on " + this.rpcServer.server.getServerName() +
-        ", is hbase.ipc.server.max.callqueue.size too small?");
-      callTooBig.sendResponseIfReady();
-      return;
-    }
-    MethodDescriptor md = null;
-    Message param = null;
-    CellScanner cellScanner = null;
-    try {
-      if (header.hasRequestParam() && header.getRequestParam()) {
-        md = this.service.getDescriptorForType().findMethodByName(
-            header.getMethodName());
-        if (md == null)
-          throw new UnsupportedOperationException(header.getMethodName());
-        builder = this.service.getRequestPrototype(md).newBuilderForType();
-        cis.resetSizeCounter();
-        int paramSize = cis.readRawVarint32();
-        offset += cis.getTotalBytesRead();
-        if (builder != null) {
-          ProtobufUtil.mergeFrom(builder, cis, paramSize);
-          param = builder.build();
-        }
-        offset += paramSize;
-      } else {
-        // currently header must have request param, so we directly throw
-        // exception here
-        String msg = "Invalid request header: "
-            + TextFormat.shortDebugString(header)
-            + ", should have param set in it";
-        RpcServer.LOG.warn(msg);
-        throw new DoNotRetryIOException(msg);
+    TextMapPropagator.Getter<RPCTInfo> getter = new TextMapPropagator.Getter<RPCTInfo>() {
+
+      @Override
+      public Iterable<String> keys(RPCTInfo carrier) {
+        return carrier.getHeadersMap().keySet();
       }
-      if (header.hasCellBlockMeta()) {
-        buf.position(offset);
-        ByteBuff dup = buf.duplicate();
-        dup.limit(offset + header.getCellBlockMeta().getLength());
-        cellScanner = this.rpcServer.cellBlockBuilder.createCellScannerReusingBuffers(
-            this.codec, this.compressionCodec, dup);
+
+      @Override
+      public String get(RPCTInfo carrier, String key) {
+        return carrier.getHeadersMap().get(key);
       }
-    } catch (Throwable t) {
-      InetSocketAddress address = this.rpcServer.getListenerAddress();
-      String msg = (address != null ? address : "(channel closed)")
-          + " is unable to read call parameter from client "
-          + getHostAddress();
-      RpcServer.LOG.warn(msg, t);
-
-      this.rpcServer.metrics.exception(t);
-
-      // probably the hbase hadoop version does not match the running hadoop
-      // version
-      if (t instanceof LinkageError) {
-        t = new DoNotRetryIOException(t);
+    };
+    Context traceCtx = GlobalOpenTelemetry.getPropagators().getTextMapPropagator()
+      .extract(Context.current(), header.getTraceInfo(), getter);
+    Span span =
+      TraceUtil.getGlobalTracer().spanBuilder("RpcServer.process").setParent(traceCtx).startSpan();
+    try (Scope scope = span.makeCurrent()) {
+      int id = header.getCallId();
+      if (RpcServer.LOG.isTraceEnabled()) {
+        RpcServer.LOG.trace("RequestHeader " + TextFormat.shortDebugString(header) +
+          " totalRequestSize: " + totalRequestSize + " bytes");
       }
-      // If the method is not present on the server, do not retry.
-      if (t instanceof UnsupportedOperationException) {
-        t = new DoNotRetryIOException(t);
+      // Enforcing the call queue size, this triggers a retry in the client
+      // This is a bit late to be doing this check - we have already read in the
+      // total request.
+      if ((totalRequestSize +
+        this.rpcServer.callQueueSizeInBytes.sum()) > this.rpcServer.maxQueueSizeInBytes) {
+        final ServerCall<?> callTooBig = createCall(id, this.service, null, null, null, null,
+          totalRequestSize, null, 0, this.callCleanup);
+        this.rpcServer.metrics.exception(RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION);
+        callTooBig.setResponse(null, null, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION,
+          "Call queue is full on " + this.rpcServer.server.getServerName() +
+            ", is hbase.ipc.server.max.callqueue.size too small?");
+        callTooBig.sendResponseIfReady();
+        return;
       }
+      MethodDescriptor md = null;
+      Message param = null;
+      CellScanner cellScanner = null;
+      try {
+        if (header.hasRequestParam() && header.getRequestParam()) {
+          md = this.service.getDescriptorForType().findMethodByName(header.getMethodName());
+          if (md == null) {
+            throw new UnsupportedOperationException(header.getMethodName());
+          }
+          builder = this.service.getRequestPrototype(md).newBuilderForType();
+          cis.resetSizeCounter();
+          int paramSize = cis.readRawVarint32();
+          offset += cis.getTotalBytesRead();
+          if (builder != null) {
+            ProtobufUtil.mergeFrom(builder, cis, paramSize);
+            param = builder.build();
+          }
+          offset += paramSize;
+        } else {
+          // currently header must have request param, so we directly throw
+          // exception here
+          String msg = "Invalid request header: " + TextFormat.shortDebugString(header) +
+            ", should have param set in it";
+          RpcServer.LOG.warn(msg);
+          throw new DoNotRetryIOException(msg);
+        }
+        if (header.hasCellBlockMeta()) {
+          buf.position(offset);
+          ByteBuff dup = buf.duplicate();
+          dup.limit(offset + header.getCellBlockMeta().getLength());
+          cellScanner = this.rpcServer.cellBlockBuilder.createCellScannerReusingBuffers(this.codec,
+            this.compressionCodec, dup);
+        }
+      } catch (Throwable t) {
+        InetSocketAddress address = this.rpcServer.getListenerAddress();
+        String msg = (address != null ? address : "(channel closed)") +
+          " is unable to read call parameter from client " + getHostAddress();
+        RpcServer.LOG.warn(msg, t);
+
+        this.rpcServer.metrics.exception(t);
+
+        // probably the hbase hadoop version does not match the running hadoop
+        // version
+        if (t instanceof LinkageError) {
+          t = new DoNotRetryIOException(t);
+        }
+        // If the method is not present on the server, do not retry.
+        if (t instanceof UnsupportedOperationException) {
+          t = new DoNotRetryIOException(t);
+        }
 
-      ServerCall<?> readParamsFailedCall = createCall(id, this.service, null, null, null, null,
-        totalRequestSize, null, 0, this.callCleanup);
-      readParamsFailedCall.setResponse(null, null, t, msg + "; " + t.getMessage());
-      readParamsFailedCall.sendResponseIfReady();
-      return;
-    }
-
-    int timeout = 0;
-    if (header.hasTimeout() && header.getTimeout() > 0) {
-      timeout = Math.max(this.rpcServer.minClientRequestTimeout, header.getTimeout());
-    }
-    ServerCall<?> call = createCall(id, this.service, md, header, param, cellScanner, totalRequestSize,
-      this.addr, timeout, this.callCleanup);
+        ServerCall<?> readParamsFailedCall = createCall(id, this.service, null, null, null, null,
+          totalRequestSize, null, 0, this.callCleanup);
+        readParamsFailedCall.setResponse(null, null, t, msg + "; " + t.getMessage());
+        readParamsFailedCall.sendResponseIfReady();
+        return;
+      }
 
-    if (!this.rpcServer.scheduler.dispatch(new CallRunner(this.rpcServer, call))) {
-      this.rpcServer.callQueueSizeInBytes.add(-1 * call.getSize());
-      this.rpcServer.metrics.exception(RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION);
-      call.setResponse(null, null, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION,
-        "Call queue is full on " + this.rpcServer.server.getServerName() +
+      int timeout = 0;
+      if (header.hasTimeout() && header.getTimeout() > 0) {
+        timeout = Math.max(this.rpcServer.minClientRequestTimeout, header.getTimeout());
+      }
+      ServerCall<?> call = createCall(id, this.service, md, header, param, cellScanner,
+        totalRequestSize, this.addr, timeout, this.callCleanup);
+
+      if (!this.rpcServer.scheduler.dispatch(new CallRunner(this.rpcServer, call))) {
+        this.rpcServer.callQueueSizeInBytes.add(-1 * call.getSize());
+        this.rpcServer.metrics.exception(RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION);
+        call.setResponse(null, null, RpcServer.CALL_QUEUE_TOO_BIG_EXCEPTION,
+          "Call queue is full on " + this.rpcServer.server.getServerName() +
             ", too many items queued ?");
-      call.sendResponseIfReady();
+        call.sendResponseIfReady();
+      }
     }
   }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index 87561ba..11978ca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -20,21 +20,28 @@ package org.apache.hadoop.hbase.ipc;
 import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE;
 import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newBlockingStub;
 import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.newStub;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.mockito.ArgumentMatchers.anyObject;
+import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 import static org.mockito.internal.verification.VerificationModeFactory.times;
 
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule;
+import io.opentelemetry.sdk.trace.data.SpanData;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
@@ -43,10 +50,12 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.util.StringUtils;
+import org.junit.Rule;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -87,6 +96,10 @@ public abstract class AbstractTestIPC {
 
   protected abstract AbstractRpcClient<?> createRpcClientNoCodec(Configuration conf);
 
+
+  @Rule
+  public OpenTelemetryRule traceRule = OpenTelemetryRule.create();
+
   /**
    * Ensure we do not HAVE TO HAVE a codec.
    */
@@ -183,7 +196,7 @@ public abstract class AbstractTestIPC {
     RpcServer rpcServer = createRpcServer(null, "testRpcServer",
         Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(
             SERVICE, null)), new InetSocketAddress("localhost", 0), CONF, scheduler);
-    verify(scheduler).init((RpcScheduler.Context) anyObject());
+    verify(scheduler).init(any(RpcScheduler.Context.class));
     try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
       rpcServer.start();
       verify(scheduler).start();
@@ -192,7 +205,7 @@ public abstract class AbstractTestIPC {
       for (int i = 0; i < 10; i++) {
         stub.echo(null, param);
       }
-      verify(scheduler, times(10)).dispatch((CallRunner) anyObject());
+      verify(scheduler, times(10)).dispatch(any(CallRunner.class));
     } finally {
       rpcServer.stop();
       verify(scheduler).stop();
@@ -427,4 +440,44 @@ public abstract class AbstractTestIPC {
     }
   }
 
+  private void assertSameTraceId() {
+    String traceId = traceRule.getSpans().get(0).getTraceId();
+    for (SpanData data : traceRule.getSpans()) {
+      // assert we are the same trace
+      assertEquals(traceId, data.getTraceId());
+    }
+  }
+
+  @Test
+  public void testTracing() throws IOException, ServiceException {
+    RpcServer rpcServer = createRpcServer(null, "testRpcServer",
+      Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)),
+      new InetSocketAddress("localhost", 0), CONF, new FifoRpcScheduler(CONF, 1));
+    try (AbstractRpcClient<?> client = createRpcClient(CONF)) {
+      rpcServer.start();
+      BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
+      stub.pause(null, PauseRequestProto.newBuilder().setMs(100).build());
+      Waiter.waitFor(CONF, 1000, () -> traceRule.getSpans().stream().map(SpanData::getName)
+        .anyMatch(s -> s.equals("RpcClient.callMethod.TestProtobufRpcProto.pause")));
+
+      assertSameTraceId();
+      for (SpanData data : traceRule.getSpans()) {
+        assertThat(
+          TimeUnit.NANOSECONDS.toMillis(data.getEndEpochNanos() - data.getStartEpochNanos()),
+          greaterThanOrEqualTo(100L));
+        assertEquals(StatusCode.OK, data.getStatus().getStatusCode());
+      }
+
+      traceRule.clearSpans();
+      assertThrows(ServiceException.class,
+        () -> stub.error(null, EmptyRequestProto.getDefaultInstance()));
+      Waiter.waitFor(CONF, 1000, () -> traceRule.getSpans().stream().map(SpanData::getName)
+        .anyMatch(s -> s.equals("RpcClient.callMethod.TestProtobufRpcProto.error")));
+
+      assertSameTraceId();
+      for (SpanData data : traceRule.getSpans()) {
+        assertEquals(StatusCode.ERROR, data.getStatus().getStatusCode());
+      }
+    }
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java
index 2601fba..c3b52a9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestNettyIPC.java
@@ -39,6 +39,7 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameter;
 import org.junit.runners.Parameterized.Parameters;
+
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoopGroup;
 import org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollSocketChannel;
diff --git a/pom.xml b/pom.xml
index 850fa54..8cf9b02 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1483,7 +1483,7 @@
     <jruby.version>9.2.13.0</jruby.version>
     <junit.version>4.13</junit.version>
     <hamcrest.version>1.3</hamcrest.version>
-    <opentelemetry.version>0.12.0</opentelemetry.version>
+    <opentelemetry.version>0.13.1</opentelemetry.version>
     <log4j.version>1.2.17</log4j.version>
     <mockito-core.version>2.28.2</mockito-core.version>
     <!--Internally we use a different version of protobuf. See hbase-protocol-shaded-->
@@ -2180,6 +2180,16 @@
         <version>${opentelemetry.version}</version>
       </dependency>
       <dependency>
+        <groupId>io.opentelemetry</groupId>
+        <artifactId>opentelemetry-sdk</artifactId>
+        <version>${opentelemetry.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>io.opentelemetry</groupId>
+        <artifactId>opentelemetry-sdk-testing</artifactId>
+        <version>${opentelemetry.version}</version>
+      </dependency>
+      <dependency>
         <groupId>com.lmax</groupId>
         <artifactId>disruptor</artifactId>
         <version>${disruptor.version}</version>

[hbase] 01/18: HBASE-26124 Backport HBASE-25373 "Remove HTrace completely in code base and try to make use of OpenTelemetry" to branch-2 (#3529)

Posted by ta...@apache.org.
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 665305cc3b50079cac62d9a55b0c5aefb13d7c67
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Thu Jul 29 09:15:10 2021 -0700

    HBASE-26124 Backport HBASE-25373 "Remove HTrace completely in code base and try to make use of OpenTelemetry" to branch-2 (#3529)
    
    1/17 commits of HBASE-22120
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 .../hadoop/hbase/io/asyncfs/AsyncFSTestBase.java   |   3 -
 hbase-client/pom.xml                               |   4 +-
 .../hbase/client/AsyncRequestFutureImpl.java       |  14 +-
 .../client/ResultBoundedCompletionService.java     |   4 +-
 .../hadoop/hbase/ipc/BlockingRpcConnection.java    |  11 +-
 .../java/org/apache/hadoop/hbase/ipc/Call.java     |   7 +-
 hbase-common/pom.xml                               |   4 +-
 .../hbase/trace/HBaseHTraceConfiguration.java      |  80 -----------
 .../hadoop/hbase/trace/SpanReceiverHost.java       | 120 -----------------
 .../org/apache/hadoop/hbase/trace/TraceUtil.java   | 105 +--------------
 hbase-external-blockcache/pom.xml                  |   4 -
 .../hadoop/hbase/io/hfile/MemcachedBlockCache.java |  14 +-
 hbase-it/pom.xml                                   |   4 +-
 .../IntegrationTestTableMapReduceUtil.java         |   1 -
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java     |  21 +--
 .../trace/IntegrationTestSendTraceRequests.java    | 135 ++++++++++---------
 hbase-mapreduce/pom.xml                            |   4 +-
 .../hadoop/hbase/mapreduce/TableMapReduceUtil.java |   1 -
 .../apache/hadoop/hbase/PerformanceEvaluation.java |  29 ++--
 hbase-protocol-shaded/pom.xml                      |   4 -
 hbase-server/pom.xml                               |   4 +-
 .../apache/hadoop/hbase/executor/EventHandler.java |  16 ++-
 .../hadoop/hbase/io/hfile/HFileReaderImpl.java     |  11 +-
 .../org/apache/hadoop/hbase/ipc/CallRunner.java    |  31 +++--
 .../org/apache/hadoop/hbase/master/HMaster.java    |   2 -
 .../hadoop/hbase/master/HMasterCommandLine.java    |   6 +-
 .../apache/hadoop/hbase/regionserver/HRegion.java  |  15 ++-
 .../hadoop/hbase/regionserver/HRegionServer.java   |   9 --
 .../regionserver/HRegionServerCommandLine.java     |   8 +-
 .../hadoop/hbase/regionserver/MemStoreFlusher.java |  14 +-
 .../hbase/regionserver/wal/AbstractFSWAL.java      |  19 ++-
 .../hadoop/hbase/regionserver/wal/AsyncFSWAL.java  |  21 +--
 .../hadoop/hbase/regionserver/wal/FSHLog.java      |  22 +--
 .../apache/hadoop/hbase/HBaseTestingUtility.java   |  13 +-
 .../hadoop/hbase/executor/TestExecutorService.java |   4 +-
 .../apache/hadoop/hbase/trace/TestHTraceHooks.java | 134 -------------------
 .../org/apache/hadoop/hbase/trace/TraceTree.java   | 148 ---------------------
 .../hadoop/hbase/wal/WALPerformanceEvaluation.java |  73 +++-------
 hbase-shaded/hbase-shaded-client/pom.xml           |   1 +
 hbase-shaded/hbase-shaded-testing-util/pom.xml     |   1 +
 hbase-shaded/pom.xml                               |   1 +
 hbase-shell/src/main/ruby/shell/commands/trace.rb  |  43 +++---
 hbase-zookeeper/pom.xml                            |   4 +
 .../hbase/zookeeper/RecoverableZooKeeper.java      |  77 ++++++++---
 pom.xml                                            |  30 ++++-
 45 files changed, 362 insertions(+), 914 deletions(-)

diff --git a/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/AsyncFSTestBase.java b/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/AsyncFSTestBase.java
index 9b276ac..fc148e8 100644
--- a/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/AsyncFSTestBase.java
+++ b/hbase-asyncfs/src/test/java/org/apache/hadoop/hbase/io/asyncfs/AsyncFSTestBase.java
@@ -19,11 +19,9 @@ package org.apache.hadoop.hbase.io.asyncfs;
 
 import java.io.File;
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
-import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -104,7 +102,6 @@ public abstract class AsyncFSTestBase {
     org.apache.log4j.Logger.getLogger(org.apache.hadoop.metrics2.impl.MetricsSystemImpl.class)
       .setLevel(org.apache.log4j.Level.ERROR);
 
-    TraceUtil.initTracer(conf);
     CLUSTER = new MiniDFSCluster.Builder(conf).numDataNodes(servers).build();
     CLUSTER.waitClusterUp();
   }
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index 08e9174..e62a7d1 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -145,8 +145,8 @@
       <artifactId>zookeeper</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-api</artifactId>
     </dependency>
     <dependency>
       <groupId>org.jruby.jcodings</groupId>
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
index d2486cc..8cfcf0c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
@@ -46,10 +46,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
-import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.htrace.core.Tracer;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -572,13 +570,9 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
       asyncProcess.incTaskCounters(multiAction.getRegions(), server);
       SingleServerRequestRunnable runnable = createSingleServerRequest(
               multiAction, numAttempt, server, callsInProgress);
-      Tracer tracer = Tracer.curThreadTracer();
 
-      if (tracer == null) {
-        return Collections.singletonList(runnable);
-      } else {
-        return Collections.singletonList(tracer.wrap(runnable, "AsyncProcess.sendMultiAction"));
-      }
+      // remove trace for runnable because HBASE-25373 and OpenTelemetry do not cover TraceRunnable
+      return Collections.singletonList(runnable);
     }
 
     // group the actions by the amount of delay
@@ -598,12 +592,10 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
     List<Runnable> toReturn = new ArrayList<>(actions.size());
     for (DelayingRunner runner : actions.values()) {
       asyncProcess.incTaskCounters(runner.getActions().getRegions(), server);
-      String traceText = "AsyncProcess.sendMultiAction";
       Runnable runnable = createSingleServerRequest(runner.getActions(), numAttempt, server, callsInProgress);
       // use a delay runner only if we need to sleep for some time
       if (runner.getSleepTime() > 0) {
         runner.setRunner(runnable);
-        traceText = "AsyncProcess.clientBackoff.sendMultiAction";
         runnable = runner;
         if (asyncProcess.connection.getConnectionMetrics() != null) {
           asyncProcess.connection.getConnectionMetrics()
@@ -614,7 +606,7 @@ class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
           asyncProcess.connection.getConnectionMetrics().incrNormalRunners();
         }
       }
-      runnable = TraceUtil.wrap(runnable, traceText);
+      // remove trace for runnable because HBASE-25373 and OpenTelemetry do not cover TraceRunnable
       toReturn.add(runnable);
 
     }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java
index 965b13c..4a96954 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java
@@ -26,7 +26,6 @@ import java.util.concurrent.RunnableFuture;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -168,7 +167,8 @@ public class ResultBoundedCompletionService<V> {
 
   public void submit(RetryingCallable<V> task, int callTimeout, int id) {
     QueueingFuture<V> newFuture = new QueueingFuture<>(task, callTimeout, id);
-    executor.execute(TraceUtil.wrap(newFuture, "ResultBoundedCompletionService.submit"));
+    // remove trace for runnable because HBASE-25373 and OpenTelemetry do not cover TraceRunnable
+    executor.execute(newFuture);
     tasks[id] = newFuture;
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
index cd8035f..1a5cb73 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcConnection.java
@@ -24,6 +24,9 @@ import static org.apache.hadoop.hbase.ipc.IPCUtil.isFatalConnectionException;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.write;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Context;
+import io.opentelemetry.context.Scope;
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
@@ -62,7 +65,6 @@ import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -593,9 +595,12 @@ class BlockingRpcConnection extends RpcConnection implements Runnable {
   }
 
   private void tracedWriteRequest(Call call) throws IOException {
-    try (TraceScope ignored = TraceUtil.createTrace("RpcClientImpl.tracedWriteRequest",
-          call.span)) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("RpcClientImpl.tracedWriteRequest")
+      .setParent(Context.current().with(call.span)).startSpan();
+    try (Scope scope = span.makeCurrent()) {
       writeRequest(call);
+    } finally {
+      span.end();
     }
   }
 
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
index 7793680..113f731 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/Call.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import io.opentelemetry.api.trace.Span;
 import java.io.IOException;
 import java.util.Optional;
 import org.apache.commons.lang3.builder.ToStringBuilder;
@@ -24,13 +25,13 @@ import org.apache.commons.lang3.builder.ToStringStyle;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.client.MetricsConnection;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.Tracer;
 import org.apache.yetus.audience.InterfaceAudience;
+
 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
 import org.apache.hbase.thirdparty.io.netty.util.Timeout;
+
 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 /** A call waiting for a value. */
@@ -73,7 +74,7 @@ class Call {
     this.timeout = timeout;
     this.priority = priority;
     this.callback = callback;
-    this.span = Tracer.getCurrentSpan();
+    this.span = Span.current();
   }
 
   /**
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index 64007c0..62661b8 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -192,8 +192,8 @@
     </dependency>
     <!-- tracing Dependencies -->
     <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-api</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/HBaseHTraceConfiguration.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/HBaseHTraceConfiguration.java
deleted file mode 100644
index 03d03d9..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/HBaseHTraceConfiguration.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * 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.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hbase.trace;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.htrace.core.HTraceConfiguration;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-@InterfaceAudience.Private
-public class HBaseHTraceConfiguration extends HTraceConfiguration {
-  private static final Logger LOG = LoggerFactory.getLogger(HBaseHTraceConfiguration.class);
-
-  public static final String KEY_PREFIX = "hbase.htrace.";
-
-  private Configuration conf;
-
-  private void handleDeprecation(String key) {
-    String oldKey = "hbase." + key;
-    String newKey = KEY_PREFIX + key;
-    String oldValue = conf.get(oldKey);
-    if (oldValue != null) {
-      LOG.warn("Warning: using deprecated configuration key " + oldKey +
-          ".  Please use " + newKey + " instead.");
-      String newValue = conf.get(newKey);
-      if (newValue == null) {
-        conf.set(newKey, oldValue);
-      } else {
-        LOG.warn("Conflicting values for " + newKey + " and " + oldKey +
-            ".  Using " + newValue);
-      }
-    }
-  }
-
-  public HBaseHTraceConfiguration(Configuration conf) {
-    this.conf = conf;
-    handleDeprecation("local-file-span-receiver.path");
-    handleDeprecation("local-file-span-receiver.capacity");
-    handleDeprecation("sampler.frequency");
-    handleDeprecation("sampler.fraction");
-    handleDeprecation("zipkin.collector-hostname");
-    handleDeprecation("zipkin.collector-port");
-    handleDeprecation("zipkin.num-threads");
-    handleDeprecation("zipkin.traced-service-hostname");
-    handleDeprecation("zipkin.traced-service-port");
-  }
-
-  @Override
-  public String get(String key) {
-    return conf.get(KEY_PREFIX + key);
-  }
-
-  @Override
-  public String get(String key, String defaultValue) {
-    return conf.get(KEY_PREFIX + key,defaultValue);
-
-  }
-
-  @Override
-  public boolean getBoolean(String key, boolean defaultValue) {
-    return conf.getBoolean(KEY_PREFIX + key, defaultValue);
-  }
-}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/SpanReceiverHost.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/SpanReceiverHost.java
deleted file mode 100644
index b967db7..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/SpanReceiverHost.java
+++ /dev/null
@@ -1,120 +0,0 @@
-/**
- * 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.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.trace;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashSet;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.htrace.core.SpanReceiver;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class provides functions for reading the names of SpanReceivers from
- * hbase-site.xml, adding those SpanReceivers to the Tracer, and closing those
- * SpanReceivers when appropriate.
- */
-@InterfaceAudience.Private
-public class SpanReceiverHost {
-  public static final String SPAN_RECEIVERS_CONF_KEY = "hbase.trace.spanreceiver.classes";
-  private static final Logger LOG = LoggerFactory.getLogger(SpanReceiverHost.class);
-  private Collection<SpanReceiver> receivers;
-  private Configuration conf;
-  private boolean closed = false;
-
-  private enum SingletonHolder {
-    INSTANCE;
-    final transient Object lock = new Object();
-    transient SpanReceiverHost host = null;
-  }
-
-  public static SpanReceiverHost getInstance(Configuration conf) {
-    synchronized (SingletonHolder.INSTANCE.lock) {
-      if (SingletonHolder.INSTANCE.host != null) {
-        return SingletonHolder.INSTANCE.host;
-      }
-
-      SpanReceiverHost host = new SpanReceiverHost(conf);
-      host.loadSpanReceivers();
-      SingletonHolder.INSTANCE.host = host;
-      return SingletonHolder.INSTANCE.host;
-    }
-
-  }
-
-  public static Configuration getConfiguration(){
-    synchronized (SingletonHolder.INSTANCE.lock) {
-      if (SingletonHolder.INSTANCE.host == null || SingletonHolder.INSTANCE.host.conf == null) {
-        return null;
-      }
-
-      return SingletonHolder.INSTANCE.host.conf;
-    }
-  }
-
-  SpanReceiverHost(Configuration conf) {
-    receivers = new HashSet<>();
-    this.conf = conf;
-  }
-
-  /**
-   * Reads the names of classes specified in the {@code hbase.trace.spanreceiver.classes} property
-   * and instantiates and registers them with the Tracer.
-   */
-  public void loadSpanReceivers() {
-    String[] receiverNames = conf.getStrings(SPAN_RECEIVERS_CONF_KEY);
-    if (receiverNames == null || receiverNames.length == 0) {
-      return;
-    }
-
-    SpanReceiver.Builder builder = new SpanReceiver.Builder(new HBaseHTraceConfiguration(conf));
-    for (String className : receiverNames) {
-      className = className.trim();
-
-      SpanReceiver receiver = builder.className(className).build();
-      if (receiver != null) {
-        receivers.add(receiver);
-        LOG.info("SpanReceiver {} was loaded successfully.", className);
-      }
-    }
-    for (SpanReceiver rcvr : receivers) {
-      TraceUtil.addReceiver(rcvr);
-    }
-  }
-
-  /**
-   * Calls close() on all SpanReceivers created by this SpanReceiverHost.
-   */
-  public synchronized void closeReceivers() {
-    if (closed) {
-      return;
-    }
-
-    closed = true;
-    for (SpanReceiver rcvr : receivers) {
-      try {
-        rcvr.close();
-      } catch (IOException e) {
-        LOG.warn("Unable to close SpanReceiver correctly: " + e.getMessage(), e);
-      }
-    }
-  }
-}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
index 10665d8..f7a111f 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
@@ -17,112 +17,19 @@
  */
 package org.apache.hadoop.hbase.trace;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.htrace.core.HTraceConfiguration;
-import org.apache.htrace.core.Sampler;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.SpanReceiver;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.trace.Tracer;
 import org.apache.yetus.audience.InterfaceAudience;
 
-/**
- * This wrapper class provides functions for accessing htrace 4+ functionality in a simplified way.
- */
 @InterfaceAudience.Private
 public final class TraceUtil {
-  private static HTraceConfiguration conf;
-  private static Tracer tracer;
-
-  private TraceUtil() {
-  }
-
-  public static void initTracer(Configuration c) {
-    if (c != null) {
-      conf = new HBaseHTraceConfiguration(c);
-    }
-
-    if (tracer == null && conf != null) {
-      tracer = new Tracer.Builder("Tracer").conf(conf).build();
-    }
-  }
-
-  /**
-   * Wrapper method to create new TraceScope with the given description
-   * @return TraceScope or null when not tracing
-   */
-  public static TraceScope createTrace(String description) {
-    return (tracer == null) ? null : tracer.newScope(description);
-  }
-
-  /**
-   * Wrapper method to create new child TraceScope with the given description
-   * and parent scope's spanId
-   * @param span parent span
-   * @return TraceScope or null when not tracing
-   */
-  public static TraceScope createTrace(String description, Span span) {
-    if (span == null) {
-      return createTrace(description);
-    }
 
-    return (tracer == null) ? null : tracer.newScope(description, span.getSpanId());
-  }
-
-  /**
-   * Wrapper method to add new sampler to the default tracer
-   * @return true if added, false if it was already added
-   */
-  public static boolean addSampler(Sampler sampler) {
-    if (sampler == null) {
-      return false;
-    }
-
-    return (tracer == null) ? false : tracer.addSampler(sampler);
-  }
-
-  /**
-   * Wrapper method to add key-value pair to TraceInfo of actual span
-   */
-  public static void addKVAnnotation(String key, String value){
-    Span span = Tracer.getCurrentSpan();
-    if (span != null) {
-      span.addKVAnnotation(key, value);
-    }
-  }
+  private static final String INSTRUMENTATION_NAME = "io.opentelemetry.contrib.hbase";
 
-  /**
-   * Wrapper method to add receiver to actual tracerpool
-   * @return true if successfull, false if it was already added
-   */
-  public static boolean addReceiver(SpanReceiver rcvr) {
-    return (tracer == null) ? false : tracer.getTracerPool().addReceiver(rcvr);
-  }
-
-  /**
-   * Wrapper method to remove receiver from actual tracerpool
-   * @return true if removed, false if doesn't exist
-   */
-  public static boolean removeReceiver(SpanReceiver rcvr) {
-    return (tracer == null) ? false : tracer.getTracerPool().removeReceiver(rcvr);
-  }
-
-  /**
-   * Wrapper method to add timeline annotiation to current span with given message
-   */
-  public static void addTimelineAnnotation(String msg) {
-    Span span = Tracer.getCurrentSpan();
-    if (span != null) {
-      span.addTimelineAnnotation(msg);
-    }
+  private TraceUtil() {
   }
 
-  /**
-   * Wrap runnable with current tracer and description
-   * @param runnable to wrap
-   * @return wrapped runnable or original runnable when not tracing
-   */
-  public static Runnable wrap(Runnable runnable, String description) {
-    return (tracer == null) ? runnable : tracer.wrap(runnable, description);
+  public static Tracer getGlobalTracer() {
+    return OpenTelemetry.getGlobalTracer(INSTRUMENTATION_NAME);
   }
 }
diff --git a/hbase-external-blockcache/pom.xml b/hbase-external-blockcache/pom.xml
index 0b37d5d..1cdffc8 100644
--- a/hbase-external-blockcache/pom.xml
+++ b/hbase-external-blockcache/pom.xml
@@ -110,10 +110,6 @@
       <artifactId>slf4j-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>
diff --git a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
index 246d7e0..350d2c7 100644
--- a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
+++ b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java
@@ -19,6 +19,8 @@
 
 package org.apache.hadoop.hbase.io.hfile;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
@@ -27,13 +29,11 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.concurrent.ExecutionException;
-
 import net.spy.memcached.CachedData;
 import net.spy.memcached.ConnectionFactoryBuilder;
 import net.spy.memcached.FailureMode;
 import net.spy.memcached.MemcachedClient;
 import net.spy.memcached.transcoders.Transcoder;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.io.ByteBuffAllocator;
@@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.apache.hadoop.hbase.nio.SingleByteBuff;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Addressing;
-import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -129,12 +128,12 @@ public class MemcachedBlockCache implements BlockCache {
   }
 
   @Override
-  public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching,
-                            boolean repeat, boolean updateCacheMetrics) {
+  public Cacheable getBlock(BlockCacheKey cacheKey, boolean caching, boolean repeat,
+    boolean updateCacheMetrics) {
     // Assume that nothing is the block cache
     HFileBlock result = null;
-
-    try (TraceScope traceScope = TraceUtil.createTrace("MemcachedBlockCache.getBlock")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("MemcachedBlockCache.getBlock").startSpan();
+    try (Scope traceScope = span.makeCurrent()) {
       result = client.get(cacheKey.toString(), tc);
     } catch (Exception e) {
       // Catch a pretty broad set of exceptions to limit any changes in the memecache client
@@ -146,6 +145,7 @@ public class MemcachedBlockCache implements BlockCache {
       }
       result = null;
     } finally {
+      span.end();
       // Update stats if this request doesn't have it turned off 100% of the time
       if (updateCacheMetrics) {
         if (result == null) {
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index d1213a0..d4ef4ec 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -247,8 +247,8 @@
       <artifactId>commons-lang3</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-api</artifactId>
     </dependency>
     <!-- Hadoop needs Netty 3.x at test scope for the minicluster -->
     <dependency>
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestTableMapReduceUtil.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestTableMapReduceUtil.java
index e21dfec..1cd4304 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestTableMapReduceUtil.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mapreduce/IntegrationTestTableMapReduceUtil.java
@@ -78,7 +78,6 @@ public class IntegrationTestTableMapReduceUtil implements Configurable, Tool {
     assertTrue(tmpjars.contains("netty"));
     assertTrue(tmpjars.contains("protobuf"));
     assertTrue(tmpjars.contains("guava"));
-    assertTrue(tmpjars.contains("htrace"));
   }
 
   @Override
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
index d946045..5e390d2 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.hbase.mttr;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assume.assumeFalse;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.concurrent.Callable;
@@ -64,9 +66,6 @@ import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.LoadTestTool;
-import org.apache.htrace.core.AlwaysSampler;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.TraceScope;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -376,12 +375,9 @@ public class IntegrationTestMTTR {
      * @param span Span.  To be kept if the time taken was over 1 second
      */
     public void addResult(long time, Span span) {
-      if (span == null) {
-        return;
-      }
       stats.addValue(TimeUnit.MILLISECONDS.convert(time, TimeUnit.NANOSECONDS));
       if (TimeUnit.SECONDS.convert(time, TimeUnit.NANOSECONDS) >= 1) {
-        traces.add(span.getTracerId());
+        traces.add(span.getSpanContext().getTraceIdAsHexString());
       }
     }
 
@@ -421,15 +417,11 @@ public class IntegrationTestMTTR {
       final int maxIterations = 10;
       int numAfterDone = 0;
       int resetCount = 0;
-      TraceUtil.addSampler(AlwaysSampler.INSTANCE);
       // Keep trying until the rs is back up and we've gotten a put through
       while (numAfterDone < maxIterations) {
         long start = System.nanoTime();
-        Span span = null;
-        try (TraceScope scope = TraceUtil.createTrace(getSpanName())) {
-          if (scope != null) {
-            span = scope.getSpan();
-          }
+        Span span = TraceUtil.getGlobalTracer().spanBuilder(getSpanName()).startSpan();
+        try (Scope scope = span.makeCurrent()) {
           boolean actionResult = doAction();
           if (actionResult && future.isDone()) {
             numAfterDone++;
@@ -460,7 +452,6 @@ public class IntegrationTestMTTR {
           throw e;
         } catch (RetriesExhaustedException e){
           throw e;
-
         // Everything else is potentially recoverable on the application side. For instance, a CM
         // action kills the RS that hosted a scanner the client was using. Continued use of that
         // scanner should be terminated, but a new scanner can be created and the read attempted
@@ -475,6 +466,8 @@ public class IntegrationTestMTTR {
             LOG.info("Too many unexpected Exceptions. Aborting.", e);
             throw e;
           }
+        } finally {
+          span.end();
         }
         result.addResult(System.nanoTime() - start, span);
       }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java
index ea219db..e9f3aa0 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java
@@ -18,10 +18,19 @@
 
 package org.apache.hadoop.hbase.trace;
 
+import io.opentelemetry.api.common.AttributeKey;
+import io.opentelemetry.api.common.Attributes;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
-import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.BufferedMutator;
@@ -31,26 +40,21 @@ import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.testclassification.IntegrationTests;
 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.htrace.core.Sampler;
-import org.apache.htrace.core.TraceScope;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
 
-import java.io.IOException;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-
 @Category(IntegrationTests.class)
 public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
-
+  private static final Logger LOG =
+    LoggerFactory.getLogger(IntegrationTestSendTraceRequests.class);
   public static final String TABLE_ARG = "t";
   public static final String CF_ARG = "f";
 
@@ -61,7 +65,6 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
   private IntegrationTestingUtility util;
   private Random random = new Random();
   private Admin admin;
-  private SpanReceiverHost receiverHost;
 
   public static void main(String[] args) throws Exception {
     Configuration configuration = HBaseConfiguration.create();
@@ -95,7 +98,6 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
   public void internalDoWork() throws Exception {
     util = createUtil();
     admin = util.getAdmin();
-    setupReceiver();
 
     deleteTable();
     createTable();
@@ -108,51 +110,53 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
     service.shutdown();
     service.awaitTermination(100, TimeUnit.SECONDS);
     Thread.sleep(90000);
-    receiverHost.closeReceivers();
     util.restoreCluster();
     util = null;
   }
 
+  @SuppressWarnings("FutureReturnValueIgnored")
   private void doScans(ExecutorService service, final LinkedBlockingQueue<Long> rks) {
+    for (int i = 0; i < 100; i++) {
+      Runnable runnable = new Runnable() {
+        private final LinkedBlockingQueue<Long> rowKeyQueue = rks;
+
+        @Override public void run() {
+          ResultScanner rs = null;
+          Span span = TraceUtil.getGlobalTracer().spanBuilder("Scan").startSpan();
+          try (Scope scope = span.makeCurrent()) {
+            Table ht = util.getConnection().getTable(tableName);
+            Scan s = new Scan();
+            s.withStartRow(Bytes.toBytes(rowKeyQueue.take()));
+            s.setBatch(7);
+            rs = ht.getScanner(s);
+            // Something to keep the jvm from removing the loop.
+            long accum = 0;
+
+            for (int x = 0; x < 1000; x++) {
+              Result r = rs.next();
+              accum |= Bytes.toLong(r.getRow());
+            }
 
-      for (int i = 0; i < 100; i++) {
-        Runnable runnable = new Runnable() {
-          private final LinkedBlockingQueue<Long> rowKeyQueue = rks;
-          @Override
-          public void run() {
-            ResultScanner rs = null;
-            TraceUtil.addSampler(Sampler.ALWAYS);
-            try (TraceScope scope = TraceUtil.createTrace("Scan")){
-              Table ht = util.getConnection().getTable(tableName);
-              Scan s = new Scan();
-              s.setStartRow(Bytes.toBytes(rowKeyQueue.take()));
-              s.setBatch(7);
-              rs = ht.getScanner(s);
-              // Something to keep the jvm from removing the loop.
-              long accum = 0;
-
-              for(int x = 0; x < 1000; x++) {
-                Result r = rs.next();
-                accum |= Bytes.toLong(r.getRow());
-              }
-
-              TraceUtil.addTimelineAnnotation("Accum result = " + accum);
+            span.addEvent("Accum result = " + accum);
 
-              ht.close();
-              ht = null;
-            } catch (IOException e) {
-              e.printStackTrace();
-              TraceUtil.addKVAnnotation("exception", e.getClass().getSimpleName());
-            } catch (Exception e) {
-            } finally {
-              if (rs != null) rs.close();
+            ht.close();
+            ht = null;
+          } catch (IOException e) {
+            LOG.warn("Exception occurred while scanning table", e);
+            span.addEvent("exception",
+              Attributes.of(AttributeKey.stringKey("exception"), e.getClass().getSimpleName()));
+          } catch (Exception e) {
+            LOG.warn("Exception occurred while scanning table", e);
+          } finally {
+            span.end();
+            if (rs != null) {
+              rs.close();
             }
-
           }
-        };
-        service.submit(runnable);
-      }
-
+        }
+      };
+      service.submit(runnable);
+    }
   }
 
   private void doGets(ExecutorService service, final LinkedBlockingQueue<Long> rowKeys)
@@ -173,9 +177,9 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
           }
 
           long accum = 0;
-          TraceUtil.addSampler(Sampler.ALWAYS);
           for (int x = 0; x < 5; x++) {
-            try (TraceScope scope = TraceUtil.createTrace("gets")) {
+            Span span = TraceUtil.getGlobalTracer().spanBuilder("gets").startSpan();
+            try (Scope scope = span.makeCurrent()) {
               long rk = rowKeyQueue.take();
               Result r1 = ht.get(new Get(Bytes.toBytes(rk)));
               if (r1 != null) {
@@ -185,10 +189,12 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
               if (r2 != null) {
                 accum |= Bytes.toLong(r2.getRow());
               }
-              TraceUtil.addTimelineAnnotation("Accum = " + accum);
+              span.addEvent("Accum = " + accum);
 
             } catch (IOException|InterruptedException ie) {
               // IGNORED
+            } finally {
+              span.end();
             }
           }
 
@@ -199,18 +205,22 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
   }
 
   private void createTable() throws IOException {
-    TraceUtil.addSampler(Sampler.ALWAYS);
-    try (TraceScope scope = TraceUtil.createTrace("createTable")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("createTable").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       util.createTable(tableName, familyName);
+    } finally {
+      span.end();
     }
   }
 
   private void deleteTable() throws IOException {
-    TraceUtil.addSampler(Sampler.ALWAYS);
-    try (TraceScope scope = TraceUtil.createTrace("deleteTable")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("deleteTable").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       if (admin.tableExists(tableName)) {
         util.deleteTable(tableName);
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -218,9 +228,9 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
     LinkedBlockingQueue<Long> rowKeys = new LinkedBlockingQueue<>(25000);
     BufferedMutator ht = util.getConnection().getBufferedMutator(this.tableName);
     byte[] value = new byte[300];
-    TraceUtil.addSampler(Sampler.ALWAYS);
     for (int x = 0; x < 5000; x++) {
-      try (TraceScope traceScope = TraceUtil.createTrace("insertData")) {
+      Span span = TraceUtil.getGlobalTracer().spanBuilder("insertData").startSpan();
+      try (Scope scope = span.makeCurrent()) {
         for (int i = 0; i < 5; i++) {
           long rk = random.nextLong();
           rowKeys.add(rk);
@@ -234,6 +244,8 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
         if ((x % 1000) == 0) {
           admin.flush(tableName);
         }
+      } finally {
+        span.end();
       }
     }
     admin.flush(tableName);
@@ -255,11 +267,4 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool {
     }
     return this.util;
   }
-
-  private void setupReceiver() {
-    Configuration conf = new Configuration(util.getConfiguration());
-    conf.setBoolean("hbase.zipkin.is-in-client-mode", true);
-
-    this.receiverHost = SpanReceiverHost.getInstance(conf);
-  }
 }
diff --git a/hbase-mapreduce/pom.xml b/hbase-mapreduce/pom.xml
index e5019f4..4f359a8 100644
--- a/hbase-mapreduce/pom.xml
+++ b/hbase-mapreduce/pom.xml
@@ -155,8 +155,8 @@
       <artifactId>slf4j-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-api</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index b844fa4..d89c5b5 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -831,7 +831,6 @@ public class TableMapReduceUtil {
       org.apache.hbase.thirdparty.io.netty.channel.Channel.class,    // hbase-shaded-netty
       org.apache.zookeeper.ZooKeeper.class,                          // zookeeper
       com.google.protobuf.Message.class,                             // protobuf
-      org.apache.htrace.core.Tracer.class,                           // htrace
       com.codahale.metrics.MetricRegistry.class,                     // metrics-core
       org.apache.commons.lang3.ArrayUtils.class);                    // commons-lang
   }
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 9cbae33..68967d6 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hbase;
 
 import com.codahale.metrics.Histogram;
 import com.codahale.metrics.UniformReservoir;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.lang.reflect.Constructor;
@@ -84,8 +86,6 @@ import org.apache.hadoop.hbase.io.hfile.RandomDistribution;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.regionserver.CompactingMemStore;
-import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration;
-import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.ByteArrayHashKey;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -104,9 +104,6 @@ import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hadoop.mapreduce.lib.reduce.LongSumReducer;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.htrace.core.ProbabilitySampler;
-import org.apache.htrace.core.Sampler;
-import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -697,6 +694,10 @@ public class PerformanceEvaluation extends Configured implements Tool {
     int totalRows = DEFAULT_ROWS_PER_GB;
     int measureAfter = 0;
     float sampleRate = 1.0f;
+    /**
+     * @deprecated Useless after switching to OpenTelemetry
+     */
+    @Deprecated
     double traceRate = 0.0;
     String tableName = TABLE_NAME;
     boolean flushCommits = true;
@@ -1147,8 +1148,6 @@ public class PerformanceEvaluation extends Configured implements Tool {
     protected final TestOptions opts;
 
     private final Status status;
-    private final Sampler traceSampler;
-    private final SpanReceiverHost receiverHost;
 
     private String testName;
     private Histogram latencyHistogram;
@@ -1170,18 +1169,9 @@ public class PerformanceEvaluation extends Configured implements Tool {
      */
     TestBase(final Configuration conf, final TestOptions options, final Status status) {
       this.conf = conf;
-      this.receiverHost = this.conf == null? null: SpanReceiverHost.getInstance(conf);
       this.opts = options;
       this.status = status;
       this.testName = this.getClass().getSimpleName();
-      if (options.traceRate >= 1.0) {
-        this.traceSampler = Sampler.ALWAYS;
-      } else if (options.traceRate > 0.0) {
-        conf.setDouble("hbase.sampler.fraction", options.traceRate);
-        this.traceSampler = new ProbabilitySampler(new HBaseHTraceConfiguration(conf));
-      } else {
-        this.traceSampler = Sampler.NEVER;
-      }
       everyN = (int) (opts.totalRows / (opts.totalRows * opts.sampleRate));
       if (options.isValueZipf()) {
         this.zipf = new RandomDistribution.Zipf(this.rand, 1, options.getValueSize(), 1.2);
@@ -1351,7 +1341,6 @@ public class PerformanceEvaluation extends Configured implements Tool {
               YammerHistogramUtils.getHistogramReport(bytesInRemoteResultsHistogram));
         }
       }
-      receiverHost.closeReceivers();
     }
 
     abstract void onTakedown() throws IOException;
@@ -1388,7 +1377,6 @@ public class PerformanceEvaluation extends Configured implements Tool {
     void testTimed() throws IOException, InterruptedException {
       int startRow = getStartRow();
       int lastRow = getLastRow();
-      TraceUtil.addSampler(traceSampler);
       // Report on completion of 1/10th of total.
       for (int ii = 0; ii < opts.cycles; ii++) {
         if (opts.cycles > 1) LOG.info("Cycle=" + ii + " of " + opts.cycles);
@@ -1396,8 +1384,11 @@ public class PerformanceEvaluation extends Configured implements Tool {
           if (i % everyN != 0) continue;
           long startTime = System.nanoTime();
           boolean requestSent = false;
-          try (TraceScope scope = TraceUtil.createTrace("test row");){
+          Span span = TraceUtil.getGlobalTracer().spanBuilder("test row").startSpan();
+          try (Scope scope = span.makeCurrent()){
             requestSent = testRow(i, startTime);
+          } finally {
+            span.end();
           }
           if ( (i - startRow) > opts.measureAfter) {
             // If multiget or multiput is enabled, say set to 10, testRow() returns immediately
diff --git a/hbase-protocol-shaded/pom.xml b/hbase-protocol-shaded/pom.xml
index 9a0b553..b9fd606 100644
--- a/hbase-protocol-shaded/pom.xml
+++ b/hbase-protocol-shaded/pom.xml
@@ -201,10 +201,6 @@
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
-    <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
-    </dependency>
   </dependencies>
   <profiles>
     <!-- Skip the tests in this module -->
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index b0ee3e7..a9a7b72 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -427,8 +427,8 @@
     </dependency>
     <!-- tracing Dependencies -->
     <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-api</artifactId>
     </dependency>
     <dependency>
       <groupId>com.lmax</groupId>
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
index df84e00..17054a5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java
@@ -18,14 +18,14 @@
  */
 package org.apache.hadoop.hbase.executor;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Context;
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.trace.TraceUtil;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -75,7 +75,7 @@ public abstract class EventHandler implements Runnable, Comparable<EventHandler>
    * Default base class constructor.
    */
   public EventHandler(Server server, EventType eventType) {
-    this.parent = Tracer.getCurrentSpan();
+    this.parent = Span.current();
     this.server = server;
     this.eventType = eventType;
     seqid = seqids.incrementAndGet();
@@ -100,10 +100,14 @@ public abstract class EventHandler implements Runnable, Comparable<EventHandler>
 
   @Override
   public void run() {
-    try (TraceScope scope = TraceUtil.createTrace(this.getClass().getSimpleName(), parent)) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder(getClass().getSimpleName())
+      .setParent(Context.current().with(parent)).startSpan();
+    try (Scope scope = span.makeCurrent()) {
       process();
-    } catch(Throwable t) {
+    } catch (Throwable t) {
       handleException(t);
+    } finally {
+      span.end();
     }
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
index 7375fa8..0bb8d23 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.io.hfile;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -48,7 +50,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.IdLock;
 import org.apache.hadoop.hbase.util.ObjectIntPair;
 import org.apache.hadoop.io.WritableUtils;
-import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -1287,7 +1288,8 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
 
     boolean useLock = false;
     IdLock.Entry lockEntry = null;
-    try (TraceScope traceScope = TraceUtil.createTrace("HFileReaderImpl.readBlock")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("HFileReaderImpl.readBlock").startSpan();
+    try (Scope traceScope = span.makeCurrent()) {
       while (true) {
         // Check cache for block. If found return.
         if (cacheConf.shouldReadBlockFromCache(expectedBlockType)) {
@@ -1302,7 +1304,7 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
             if (LOG.isTraceEnabled()) {
               LOG.trace("From Cache " + cachedBlock);
             }
-            TraceUtil.addTimelineAnnotation("blockCacheHit");
+            span.addEvent("blockCacheHit");
             assert cachedBlock.isUnpacked() : "Packed block leak.";
             if (cachedBlock.getBlockType().isData()) {
               if (updateCacheMetrics) {
@@ -1332,7 +1334,7 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
           // Carry on, please load.
         }
 
-        TraceUtil.addTimelineAnnotation("blockCacheMiss");
+        span.addEvent("blockCacheMiss");
         // Load block from filesystem.
         HFileBlock hfileBlock = fsBlockReader.readBlockData(dataBlockOffset, onDiskBlockSize, pread,
           !isCompaction, shouldUseHeap(expectedBlockType));
@@ -1362,6 +1364,7 @@ public abstract class HFileReaderImpl implements HFile.Reader, Configurable {
       if (lockEntry != null) {
         offsetLock.releaseLockEntry(lockEntry);
       }
+      span.end();
     }
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index e5354d7..3ae089e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -17,23 +17,24 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.net.InetSocketAddress;
 import java.nio.channels.ClosedChannelException;
 import java.util.Optional;
-
 import org.apache.hadoop.hbase.CallDroppedException;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
-import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
+
 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
 
 /**
@@ -94,6 +95,14 @@ public class CallRunner {
     this.rpcServer = null;
   }
 
+  private String getServiceName() {
+    return call.getService() != null ? call.getService().getDescriptorForType().getName() : "";
+  }
+
+  private String getMethodName() {
+    return call.getMethod() != null ? call.getMethod().getName() : "";
+  }
+
   public void run() {
     try {
       if (call.disconnectSince() >= 0) {
@@ -118,18 +127,16 @@ public class CallRunner {
       String error = null;
       Pair<Message, CellScanner> resultPair = null;
       RpcServer.CurCall.set(call);
-      TraceScope traceScope = null;
-      try {
+      String serviceName = getServiceName();
+      String methodName = getMethodName();
+      String traceString = serviceName + "." + methodName;
+      Span span = TraceUtil.getGlobalTracer().spanBuilder(traceString).startSpan();
+      try (Scope traceScope = span.makeCurrent()) {
         if (!this.rpcServer.isStarted()) {
           InetSocketAddress address = rpcServer.getListenerAddress();
           throw new ServerNotRunningYetException("Server " +
               (address != null ? address : "(channel closed)") + " is not running yet");
         }
-        String serviceName =
-            call.getService() != null ? call.getService().getDescriptorForType().getName() : "";
-        String methodName = (call.getMethod() != null) ? call.getMethod().getName() : "";
-        String traceString = serviceName + "." + methodName;
-        traceScope = TraceUtil.createTrace(traceString);
         // make the call
         resultPair = this.rpcServer.call(call, this.status);
       } catch (TimeoutIOException e){
@@ -151,14 +158,12 @@ public class CallRunner {
           throw (Error)e;
         }
       } finally {
-        if (traceScope != null) {
-          traceScope.close();
-        }
         RpcServer.CurCall.set(null);
         if (resultPair != null) {
           this.rpcServer.addCallSize(call.getSize() * -1);
           sucessful = true;
         }
+        span.end();
       }
       this.status.markComplete("To send response");
       // return back the RPC request read BB we can do here. It is done by now.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index e6ab627..350e68a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -201,7 +201,6 @@ import org.apache.hadoop.hbase.replication.regionserver.ReplicationStatus;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.SecurityConstants;
 import org.apache.hadoop.hbase.security.UserProvider;
-import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -423,7 +422,6 @@ public class HMaster extends HRegionServer implements MasterServices {
    */
   public HMaster(final Configuration conf) throws IOException {
     super(conf);
-    TraceUtil.initTracer(conf);
     try {
       if (conf.getBoolean(MAINTENANCE_MODE, false)) {
         LOG.info("Detected {}=true via configuration.", MAINTENANCE_MODE);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java
index 6c3ee1d..0f0a2b6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java
@@ -21,15 +21,12 @@ package org.apache.hadoop.hbase.master;
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.MasterNotRunningException;
 import org.apache.hadoop.hbase.ZNodeClearer;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.trace.TraceUtil;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -39,6 +36,7 @@ import org.apache.hadoop.hbase.util.ServerCommandLine;
 import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -167,8 +165,6 @@ public class HMasterCommandLine extends ServerCommandLine {
 
   private int startMaster() {
     Configuration conf = getConf();
-    TraceUtil.initTracer(conf);
-
     try {
       // If 'local', defer to LocalHBaseCluster instance.  Starts master
       // and regionserver both in the one JVM.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 20c52c3..dac4e02 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -21,6 +21,8 @@ import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
 import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
 import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
 import edu.umd.cs.findbugs.annotations.Nullable;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -191,7 +193,6 @@ import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALSplitUtil;
 import org.apache.hadoop.hbase.wal.WALSplitUtil.MutationReplay;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -6573,8 +6574,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     RowLockImpl result = null;
 
     boolean success = false;
-    try (TraceScope scope = TraceUtil.createTrace("HRegion.getRowLock")) {
-      TraceUtil.addTimelineAnnotation("Getting a " + (readLock?"readLock":"writeLock"));
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("HRegion.getRowLock").startSpan();
+    try (Scope scope = span.makeCurrent()) {
+      span.addEvent("Getting a " + (readLock ? "readLock" : "writeLock"));
       // Keep trying until we have a lock or error out.
       // TODO: do we need to add a time component here?
       while (result == null) {
@@ -6611,7 +6613,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       }
 
       if (timeout <= 0 || !result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS)) {
-        TraceUtil.addTimelineAnnotation("Failed to get row lock");
+        span.addEvent("Failed to get row lock");
         String message = "Timed out waiting for lock for row: " + rowKey + " in region "
             + getRegionInfo().getEncodedName();
         if (reachDeadlineFirst) {
@@ -6629,7 +6631,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey,
           getRegionInfo().getRegionNameAsString());
       }
-      TraceUtil.addTimelineAnnotation("Interrupted exception getting row lock");
+      span.addEvent("Interrupted exception getting row lock");
       throw throwOnInterrupt(ie);
     } catch (Error error) {
       // The maximum lock count for read lock is 64K (hardcoded), when this maximum count
@@ -6638,13 +6640,14 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       LOG.warn("Error to get row lock for {}, in region {}, cause: {}", Bytes.toStringBinary(row),
         getRegionInfo().getRegionNameAsString(), error);
       IOException ioe = new IOException(error);
-      TraceUtil.addTimelineAnnotation("Error getting row lock");
+      span.addEvent("Error getting row lock");
       throw ioe;
     } finally {
       // Clean up the counts just in case this was the thing keeping the context alive.
       if (!success && rowLockContext != null) {
         rowLockContext.cleanUp();
       }
+      span.end();
     }
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 5bfdc13..7b459de 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -159,8 +159,6 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
 import org.apache.hadoop.hbase.security.access.AccessChecker;
 import org.apache.hadoop.hbase.security.access.ZKPermissionWatcher;
-import org.apache.hadoop.hbase.trace.SpanReceiverHost;
-import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
@@ -398,7 +396,6 @@ public class HRegionServer extends Thread implements
 
   private MetricsRegionServer metricsRegionServer;
   MetricsRegionServerWrapperImpl metricsRegionServerImpl;
-  private SpanReceiverHost spanReceiverHost;
 
   /**
    * ChoreService used to schedule tasks that we want to run periodically
@@ -595,7 +592,6 @@ public class HRegionServer extends Thread implements
    */
   public HRegionServer(final Configuration conf) throws IOException {
     super("RegionServer");  // thread name
-    TraceUtil.initTracer(conf);
     try {
       this.startcode = EnvironmentEdgeManager.currentTime();
       this.conf = conf;
@@ -667,7 +663,6 @@ public class HRegionServer extends Thread implements
         (t, e) -> abort("Uncaught exception in executorService thread " + t.getName(), e);
 
       initializeFileSystem();
-      spanReceiverHost = SpanReceiverHost.getInstance(getConfiguration());
 
       this.configurationManager = new ConfigurationManager();
       setupWindows(getConfiguration(), getConfigurationManager());
@@ -2714,10 +2709,6 @@ public class HRegionServer extends Thread implements
     if (this.cacheFlusher != null) {
       this.cacheFlusher.join();
     }
-
-    if (this.spanReceiverHost != null) {
-      this.spanReceiverHost.closeReceivers();
-    }
     if (this.walRoller != null) {
       this.walRoller.close();
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServerCommandLine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServerCommandLine.java
index afd85f8..5fd1233 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServerCommandLine.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServerCommandLine.java
@@ -18,14 +18,13 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
-import org.apache.hadoop.hbase.trace.TraceUtil;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.LocalHBaseCluster;
 import org.apache.hadoop.hbase.util.ServerCommandLine;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Class responsible for parsing the command line and starting the
@@ -51,7 +50,6 @@ public class HRegionServerCommandLine extends ServerCommandLine {
 
   private int start() throws Exception {
     Configuration conf = getConf();
-    TraceUtil.initTracer(conf);
     try {
       // If 'local', don't start a region server here. Defer to
       // LocalHBaseCluster. It manages 'local' clusters.
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
index 77755bd..0856357 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java
@@ -18,6 +18,8 @@
  */
 package org.apache.hadoop.hbase.regionserver;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.ArrayList;
@@ -36,7 +38,6 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.LongAdder;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.DroppedSnapshotException;
 import org.apache.hadoop.hbase.HConstants;
@@ -49,12 +50,12 @@ import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
 /**
  * Thread that flushes cache on request
  *
@@ -712,10 +713,12 @@ public class MemStoreFlusher implements FlushRequester {
    * amount of memstore consumption.
    */
   public void reclaimMemStoreMemory() {
-    try (TraceScope scope = TraceUtil.createTrace("MemStoreFluser.reclaimMemStoreMemory")) {
+    Span span =
+      TraceUtil.getGlobalTracer().spanBuilder("MemStoreFluser.reclaimMemStoreMemory").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       FlushType flushType = isAboveHighWaterMark();
       if (flushType != FlushType.NORMAL) {
-        TraceUtil.addTimelineAnnotation("Force Flush. We're above high water mark.");
+        span.addEvent("Force Flush. We're above high water mark.");
         long start = EnvironmentEdgeManager.currentTime();
         long nextLogTimeMs = start;
         synchronized (this.blockSignal) {
@@ -784,6 +787,7 @@ public class MemStoreFlusher implements FlushRequester {
         if (flushType != FlushType.NORMAL) {
           wakeupFlushThread();
         }
+        span.end();
       }
     }
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 0d9c140..5c247d1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -25,6 +25,8 @@ import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.c
 import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull;
 
 import com.lmax.disruptor.RingBuffer;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -83,7 +85,6 @@ import org.apache.hadoop.hbase.wal.WALProvider.WriterBase;
 import org.apache.hadoop.hbase.wal.WALSplitter;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -784,9 +785,12 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
    * @throws IOException if there is a problem flushing or closing the underlying FS
    */
   Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException {
-    try (TraceScope scope = TraceUtil.createTrace("FSHFile.replaceWriter")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHFile.replaceWriter").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       doReplaceWriter(oldPath, newPath, nextWriter);
       return newPath;
+    } finally {
+      span.end();
     }
   }
 
@@ -834,7 +838,8 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
         LOG.debug("WAL closed. Skipping rolling of writer");
         return regionsToFlush;
       }
-      try (TraceScope scope = TraceUtil.createTrace("FSHLog.rollWriter")) {
+      Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHLog.rollWriter").startSpan();
+      try (Scope scope = span.makeCurrent()) {
         Path oldPath = getOldPath();
         Path newPath = getNewPath();
         // Any exception from here on is catastrophic, non-recoverable so we currently abort.
@@ -861,6 +866,8 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
         throw new IOException(
             "Underlying FileSystem can't meet stream requirements. See RS log " + "for details.",
             exception);
+      } finally {
+        span.end();
       }
       return regionsToFlush;
     } finally {
@@ -1052,7 +1059,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
           .append(TimeUnit.NANOSECONDS.toMillis(timeInNanos))
           .append(" ms, current pipeline: ")
           .append(Arrays.toString(getPipeline())).toString();
-      TraceUtil.addTimelineAnnotation(msg);
+      Span.current().addEvent(msg);
       LOG.info(msg);
       // A single sync took too long.
       // Elsewhere in checkSlowSync, called from checkLogRoll, we will look at cumulative
@@ -1088,12 +1095,14 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     long txid = txidHolder.longValue();
     ServerCall<?> rpcCall = RpcServer.getCurrentCall().filter(c -> c instanceof ServerCall)
       .filter(c -> c.getCellScanner() != null).map(c -> (ServerCall) c).orElse(null);
-    try (TraceScope scope = TraceUtil.createTrace(implClassName + ".append")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder(implClassName + ".append").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore, rpcCall);
       entry.stampRegionSequenceId(we);
       ringBuffer.get(txid).load(entry);
     } finally {
       ringBuffer.publish(txid);
+      span.end();
     }
     return txid;
   }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index ae26a47..eef0575 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -20,10 +20,11 @@ package org.apache.hadoop.hbase.regionserver.wal;
 import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.ERROR;
 import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SIZE;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
-
 import com.lmax.disruptor.RingBuffer;
 import com.lmax.disruptor.Sequence;
 import com.lmax.disruptor.Sequencer;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.util.ArrayDeque;
@@ -44,7 +45,6 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Supplier;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -58,12 +58,11 @@ import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
 import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hbase.thirdparty.io.netty.channel.Channel;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoop;
 import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
@@ -401,7 +400,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   }
 
   private void addTimeAnnotation(SyncFuture future, String annotation) {
-    TraceUtil.addTimelineAnnotation(annotation);
+    Span.current().addEvent(annotation);
     // TODO handle htrace API change, see HBASE-18895
     // future.setSpan(scope.getSpan());
   }
@@ -624,7 +623,8 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
 
   @Override
   public void sync(boolean forceSync) throws IOException {
-    try (TraceScope scope = TraceUtil.createTrace("AsyncFSWAL.sync")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("AsyncFSWAL.sync").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       long txid = waitingConsumePayloads.next();
       SyncFuture future;
       try {
@@ -638,6 +638,8 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
         consumeExecutor.execute(consumer);
       }
       blockOnSync(future);
+    } finally {
+      span.end();
     }
   }
 
@@ -646,7 +648,8 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     if (highestSyncedTxid.get() >= txid) {
       return;
     }
-    try (TraceScope scope = TraceUtil.createTrace("AsyncFSWAL.sync")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("AsyncFSWAL.sync").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       // here we do not use ring buffer sequence as txid
       long sequence = waitingConsumePayloads.next();
       SyncFuture future;
@@ -661,6 +664,8 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
         consumeExecutor.execute(consumer);
       }
       blockOnSync(future);
+    } finally {
+      span.end();
     }
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 690f545..762f1a1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -29,6 +29,8 @@ import com.lmax.disruptor.LifecycleAware;
 import com.lmax.disruptor.TimeoutException;
 import com.lmax.disruptor.dsl.Disruptor;
 import com.lmax.disruptor.dsl.ProducerType;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Arrays;
@@ -59,7 +61,6 @@ import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.htrace.core.TraceScope;
 import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -364,7 +365,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
           // use assert to make sure no change breaks the logic that
           // sequence and zigzagLatch will be set together
           assert sequence > 0L : "Failed to get sequence from ring buffer";
-          TraceUtil.addTimelineAnnotation("awaiting safepoint");
+          Span.current().addEvent("awaiting safepoint");
           syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer(sequence, false));
         }
       } catch (FailedSyncBeforeLogCloseException e) {
@@ -436,10 +437,11 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   }
 
   private void closeWriter(Writer writer, Path path, boolean syncCloseCall) throws IOException {
+    Span span = Span.current();
     try {
-      TraceUtil.addTimelineAnnotation("closing writer");
+      span.addEvent("closing writer");
       writer.close();
-      TraceUtil.addTimelineAnnotation("writer closed");
+      span.addEvent("writer closed");
     } catch (IOException ioe) {
       int errors = closeErrorCount.incrementAndGet();
       boolean hasUnflushedEntries = isUnflushedEntries();
@@ -649,10 +651,10 @@ public class FSHLog extends AbstractFSWAL<Writer> {
           long start = System.nanoTime();
           Throwable lastException = null;
           try {
-            TraceUtil.addTimelineAnnotation("syncing writer");
+            Span.current().addEvent("syncing writer");
             long unSyncedFlushSeq = highestUnsyncedTxid;
             writer.sync(sf.isForceSync());
-            TraceUtil.addTimelineAnnotation("writer synced");
+            Span.current().addEvent("writer synced");
             if (unSyncedFlushSeq > currentSequence) {
               currentSequence = unSyncedFlushSeq;
             }
@@ -791,7 +793,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   }
 
   // Sync all known transactions
-  private void publishSyncThenBlockOnCompletion(TraceScope scope, boolean forceSync) throws IOException {
+  private void publishSyncThenBlockOnCompletion(Scope scope, boolean forceSync) throws IOException {
     SyncFuture syncFuture = publishSyncOnRingBuffer(forceSync);
     blockOnSync(syncFuture);
   }
@@ -823,7 +825,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
 
   @Override
   public void sync(boolean forceSync) throws IOException {
-    try (TraceScope scope = TraceUtil.createTrace("FSHLog.sync")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHLog.sync").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       publishSyncThenBlockOnCompletion(scope, forceSync);
     }
   }
@@ -839,7 +842,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
       // Already sync'd.
       return;
     }
-    try (TraceScope scope = TraceUtil.createTrace("FSHLog.sync")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHLog.sync").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       publishSyncThenBlockOnCompletion(scope, forceSync);
     }
   }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 1c4636e..f67fece 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import edu.umd.cs.findbugs.annotations.Nullable;
 import java.io.File;
 import java.io.IOException;
@@ -51,8 +50,8 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
-import java.util.stream.Collectors;
 import java.util.function.BooleanSupplier;
+import java.util.stream.Collectors;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -119,12 +118,10 @@ import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.security.HBaseKerberosUtils;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.visibility.VisibilityLabelsCache;
-import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
-import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.JVMClusterUtil;
 import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
@@ -147,11 +144,12 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MiniMRCluster;
 import org.apache.hadoop.mapred.TaskLog;
 import org.apache.hadoop.minikdc.MiniKdc;
-import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper.States;
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
 
 /**
  * Facility for testing HBase. Replacement for
@@ -663,8 +661,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     Log4jUtils.setLogLevel(org.apache.hadoop.metrics2.impl.MetricsSystemImpl.class.getName(),
       "ERROR");
 
-    TraceUtil.initTracer(conf);
-
     this.dfsCluster = new MiniDFSCluster(0, this.conf, servers, true, true,
         true, null, racks, hosts, null);
 
@@ -1172,7 +1168,6 @@ public class HBaseTestingUtility extends HBaseZKTestingUtility {
     Log4jUtils.setLogLevel(org.apache.hadoop.hbase.ScheduledChore.class.getName(), "INFO");
 
     Configuration c = new Configuration(this.conf);
-    TraceUtil.initTracer(c);
     this.hbaseCluster = new MiniHBaseCluster(c, option.getNumMasters(),
       option.getNumAlwaysStandByMasters(), option.getNumRegionServers(), option.getRsPorts(),
       option.getMasterClass(), option.getRsClass());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java
index 6b58d07..5df089b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/executor/TestExecutorService.java
@@ -165,8 +165,8 @@ public class TestExecutorService {
     private final AtomicBoolean lock;
     private AtomicInteger counter;
 
-    public TestEventHandler(Server server, EventType eventType,
-                            AtomicBoolean lock, AtomicInteger counter) {
+    public TestEventHandler(Server server, EventType eventType, AtomicBoolean lock,
+      AtomicInteger counter) {
       super(server, eventType);
       this.lock = lock;
       this.counter = counter;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java
deleted file mode 100644
index b1fc5b9..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/**
- * 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.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.trace;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Collection;
-import java.util.LinkedList;
-import java.util.List;
-import org.apache.hadoop.hbase.HBaseClassTestRule;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.StartMiniClusterOption;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.MiscTests;
-import org.apache.htrace.core.POJOSpanReceiver;
-import org.apache.htrace.core.Sampler;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.TraceScope;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.ClassRule;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TestName;
-
-import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
-
-@Ignore // We don't support htrace in hbase-2.0.0 and this flakey is a little flakey.
-@Category({MiscTests.class, MediumTests.class})
-public class TestHTraceHooks {
-
-  @ClassRule
-  public static final HBaseClassTestRule CLASS_RULE =
-      HBaseClassTestRule.forClass(TestHTraceHooks.class);
-
-  private static final byte[] FAMILY_BYTES = "family".getBytes();
-  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-  private static POJOSpanReceiver rcvr;
-
-  @Rule
-  public TestName name = new TestName();
-
-  @BeforeClass
-  public static void before() throws Exception {
-    StartMiniClusterOption option = StartMiniClusterOption.builder()
-        .numMasters(2).numRegionServers(3).numDataNodes(3).build();
-    TEST_UTIL.startMiniCluster(option);
-    rcvr = new POJOSpanReceiver(new HBaseHTraceConfiguration(TEST_UTIL.getConfiguration()));
-    TraceUtil.addReceiver(rcvr);
-    TraceUtil.addSampler(new Sampler() {
-      @Override
-      public boolean next() {
-        return true;
-      }
-    });
-  }
-
-  @AfterClass
-  public static void after() throws Exception {
-    TEST_UTIL.shutdownMiniCluster();
-    TraceUtil.removeReceiver(rcvr);
-    rcvr = null;
-  }
-
-  @Test
-  public void testTraceCreateTable() throws Exception {
-    Table table;
-    Span createTableSpan;
-    try (TraceScope scope = TraceUtil.createTrace("creating table")) {
-      createTableSpan = scope.getSpan();
-      table = TEST_UTIL.createTable(TableName.valueOf(name.getMethodName()), FAMILY_BYTES);
-    }
-
-    // Some table creation is async.  Need to make sure that everything is full in before
-    // checking to see if the spans are there.
-    TEST_UTIL.waitFor(10000, new Waiter.Predicate<Exception>() {
-      @Override public boolean evaluate() throws Exception {
-        return (rcvr == null) ? true : rcvr.getSpans().size() >= 5;
-      }
-    });
-
-    Collection<Span> spans = Sets.newHashSet(rcvr.getSpans());
-    List<Span> roots = new LinkedList<>();
-    TraceTree traceTree = new TraceTree(spans);
-    roots.addAll(traceTree.getSpansByParent().find(createTableSpan.getSpanId()));
-
-    // Roots was made 3 in hbase2. It used to be 1. We changed it back to 1 on upgrade to
-    // htrace-4.2 just to get the test to pass (traces are not wholesome in hbase2; TODO).
-    assertEquals(1, roots.size());
-    assertEquals("creating table", createTableSpan.getDescription());
-
-    if (spans != null) {
-      assertTrue(spans.size() > 5);
-    }
-
-    Put put = new Put("row".getBytes());
-    put.addColumn(FAMILY_BYTES, "col".getBytes(), "value".getBytes());
-
-    Span putSpan;
-
-    try (TraceScope scope = TraceUtil.createTrace("doing put")) {
-      putSpan = scope.getSpan();
-      table.put(put);
-    }
-
-    spans = rcvr.getSpans();
-    traceTree = new TraceTree(spans);
-    roots.clear();
-    roots.addAll(traceTree.getSpansByParent().find(putSpan.getSpanId()));
-    assertEquals(1, roots.size());
-  }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TraceTree.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TraceTree.java
deleted file mode 100644
index eb209d0..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TraceTree.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/**
- * 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.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.trace;
-
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.SpanId;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.TreeSet;
-
-/**
- * Used to create the graph formed by spans.
- */
-public class TraceTree {
-
-  public static class SpansByParent {
-    private static Comparator<Span> COMPARATOR =
-        new Comparator<Span>() {
-          @Override
-          public int compare(Span a, Span b) {
-            return a.getSpanId().compareTo(b.getSpanId());
-          }
-        };
-
-    private final TreeSet<Span> treeSet;
-
-    private final HashMap<SpanId, LinkedList<Span>> parentToSpans;
-
-    SpansByParent(Collection<Span> spans) {
-      TreeSet<Span> treeSet = new TreeSet<Span>(COMPARATOR);
-      parentToSpans = new HashMap<SpanId, LinkedList<Span>>();
-      for (Span span : spans) {
-        treeSet.add(span);
-        for (SpanId parent : span.getParents()) {
-          LinkedList<Span> list = parentToSpans.get(parent);
-          if (list == null) {
-            list = new LinkedList<Span>();
-            parentToSpans.put(parent, list);
-          }
-          list.add(span);
-        }
-        if (span.getParents().length == 0) {
-          LinkedList<Span> list = parentToSpans.get(SpanId.INVALID);
-          if (list == null) {
-            list = new LinkedList<Span>();
-            parentToSpans.put(SpanId.INVALID, list);
-          }
-          list.add(span);
-        }
-      }
-      this.treeSet = treeSet;
-    }
-
-    public List<Span> find(SpanId parentId) {
-      LinkedList<Span> spans = parentToSpans.get(parentId);
-      if (spans == null) {
-        return new LinkedList<Span>();
-      }
-      return spans;
-    }
-
-    public Iterator<Span> iterator() {
-      return Collections.unmodifiableSortedSet(treeSet).iterator();
-    }
-  }
-
-  public static class SpansByProcessId {
-    private static Comparator<Span> COMPARATOR =
-        new Comparator<Span>() {
-          @Override
-          public int compare(Span a, Span b) {
-            return a.getSpanId().compareTo(b.getSpanId());
-          }
-        };
-
-    private final TreeSet<Span> treeSet;
-
-    SpansByProcessId(Collection<Span> spans) {
-      TreeSet<Span> treeSet = new TreeSet<Span>(COMPARATOR);
-      for (Span span : spans) {
-        treeSet.add(span);
-      }
-      this.treeSet = treeSet;
-    }
-
-    public Iterator<Span> iterator() {
-      return Collections.unmodifiableSortedSet(treeSet).iterator();
-    }
-  }
-
-  private final SpansByParent spansByParent;
-  private final SpansByProcessId spansByProcessId;
-
-  /**
-   * Create a new TraceTree
-   *
-   * @param spans The collection of spans to use to create this TraceTree. Should
-   *              have at least one root span.
-   */
-  public TraceTree(Collection<Span> spans) {
-    if (spans == null) {
-      spans = Collections.emptySet();
-    }
-    this.spansByParent = new SpansByParent(spans);
-    this.spansByProcessId = new SpansByProcessId(spans);
-  }
-
-  public SpansByParent getSpansByParent() {
-    return spansByParent;
-  }
-
-  public SpansByProcessId getSpansByProcessId() {
-    return spansByProcessId;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder bld = new StringBuilder();
-    String prefix = "";
-    for (Iterator<Span> iter = spansByParent.iterator(); iter.hasNext();) {
-      Span span = iter.next();
-      bld.append(prefix).append(span.toString());
-      prefix = "\n";
-    }
-    return bld.toString();
-  }
-}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
index 9f02d2e..6b61751 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java
@@ -25,6 +25,8 @@ import com.codahale.metrics.Histogram;
 import com.codahale.metrics.Meter;
 import com.codahale.metrics.MetricFilter;
 import com.codahale.metrics.MetricRegistry;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -58,8 +60,6 @@ import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl;
 import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader;
 import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter;
 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
-import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration;
-import org.apache.hadoop.hbase.trace.SpanReceiverHost;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
@@ -68,10 +68,6 @@ import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.htrace.core.ProbabilitySampler;
-import org.apache.htrace.core.Sampler;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -129,12 +125,10 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
     private final boolean noSync;
     private final HRegion region;
     private final int syncInterval;
-    private final Sampler loopSampler;
     private final NavigableMap<byte[], Integer> scopes;
 
     WALPutBenchmark(final HRegion region, final TableDescriptor htd,
-        final long numIterations, final boolean noSync, final int syncInterval,
-        final double traceFreq) {
+        final long numIterations, final boolean noSync, final int syncInterval) {
       this.numIterations = numIterations;
       this.noSync = noSync;
       this.syncInterval = syncInterval;
@@ -144,24 +138,6 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
       for(byte[] fam : htd.getColumnFamilyNames()) {
         scopes.put(fam, 0);
       }
-      String spanReceivers = getConf().get("hbase.trace.spanreceiver.classes");
-      if (spanReceivers == null || spanReceivers.isEmpty()) {
-        loopSampler = Sampler.NEVER;
-      } else {
-        if (traceFreq <= 0.0) {
-          LOG.warn("Tracing enabled but traceFreq=0.");
-          loopSampler = Sampler.NEVER;
-        } else if (traceFreq >= 1.0) {
-          loopSampler = Sampler.ALWAYS;
-          if (numIterations > 1000) {
-            LOG.warn("Full tracing of all iterations will produce a lot of data. Be sure your"
-              + " SpanReceiver can keep up.");
-          }
-        } else {
-          getConf().setDouble("hbase.sampler.fraction", traceFreq);
-          loopSampler = new ProbabilitySampler(new HBaseHTraceConfiguration(getConf()));
-        }
-      }
     }
 
     @Override
@@ -170,13 +146,14 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
       byte[] value = new byte[valueSize];
       Random rand = new Random(Thread.currentThread().getId());
       WAL wal = region.getWAL();
-
-      try (TraceScope threadScope = TraceUtil.createTrace("WALPerfEval." + Thread.currentThread().getName())) {
+      Span threadSpan = TraceUtil.getGlobalTracer()
+        .spanBuilder("WALPerfEval." + Thread.currentThread().getName()).startSpan();
+      try (Scope threadScope = threadSpan.makeCurrent()) {
         int lastSync = 0;
-        TraceUtil.addSampler(loopSampler);
         for (int i = 0; i < numIterations; ++i) {
-          assert Tracer.getCurrentSpan() == threadScope.getSpan() : "Span leak detected.";
-          try (TraceScope loopScope = TraceUtil.createTrace("runLoopIter" + i)) {
+          assert Span.current() == threadSpan : "Span leak detected.";
+          Span loopSpan = TraceUtil.getGlobalTracer().spanBuilder("runLoopIter" + i).startSpan();
+          try (Scope loopScope = loopSpan.makeCurrent()) {
             long now = System.nanoTime();
             Put put = setupPut(rand, key, value, numFamilies);
             WALEdit walEdit = new WALEdit();
@@ -192,10 +169,14 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
               }
             }
             latencyHistogram.update(System.nanoTime() - now);
+          } finally {
+            loopSpan.end();
           }
         }
       } catch (Exception e) {
         LOG.error(getClass().getSimpleName() + " Thread failed", e);
+      } finally {
+        threadSpan.end();
       }
     }
   }
@@ -216,9 +197,6 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
     boolean compress = false;
     String cipher = null;
     int numRegions = 1;
-    String spanReceivers = getConf().get("hbase.trace.spanreceiver.classes");
-    boolean trace = spanReceivers != null && !spanReceivers.isEmpty();
-    double traceFreq = 1.0;
     // Process command line args
     for (int i = 0; i < args.length; i++) {
       String cmd = args[i];
@@ -258,7 +236,8 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
         } else if (cmd.equals("-regions")) {
           numRegions = Integer.parseInt(args[++i]);
         } else if (cmd.equals("-traceFreq")) {
-          traceFreq = Double.parseDouble(args[++i]);
+          // keep it here for compatible
+          System.err.println("-traceFreq is not supported any more");
         } else if (cmd.equals("-h")) {
           printUsageAndExit();
         } else if (cmd.equals("--help")) {
@@ -307,13 +286,8 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
     CommonFSUtils.setFsDefault(getConf(), CommonFSUtils.getRootDir(getConf()));
     FileSystem fs = FileSystem.get(getConf());
     LOG.info("FileSystem={}, rootDir={}", fs, rootRegionDir);
-
-    SpanReceiverHost receiverHost = trace ? SpanReceiverHost.getInstance(getConf()) : null;
-    final Sampler sampler = trace ? Sampler.ALWAYS : Sampler.NEVER;
-    TraceUtil.addSampler(sampler);
-    TraceScope scope = TraceUtil.createTrace("WALPerfEval");
-
-    try {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("WALPerfEval").startSpan();
+    try (Scope scope = span.makeCurrent()){
       rootRegionDir = rootRegionDir.makeQualified(fs.getUri(), fs.getWorkingDirectory());
       cleanRegionRootDir(fs, rootRegionDir);
       CommonFSUtils.setRootDir(getConf(), rootRegionDir);
@@ -330,8 +304,8 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
           // a table per desired region means we can avoid carving up the key space
           final TableDescriptor htd = createHTableDescriptor(i, numFamilies);
           regions[i] = openRegion(fs, rootRegionDir, htd, wals, roll, roller);
-          benchmarks[i] = TraceUtil.wrap(new WALPutBenchmark(regions[i], htd, numIterations, noSync,
-              syncInterval, traceFreq), "");
+          benchmarks[i] =
+            new WALPutBenchmark(regions[i], htd, numIterations, noSync, syncInterval);
         }
         ConsoleReporter reporter = ConsoleReporter.forRegistry(metrics).
           outputTo(System.out).convertRatesTo(TimeUnit.SECONDS).filter(MetricFilter.ALL).build();
@@ -380,19 +354,14 @@ public final class WALPerformanceEvaluation extends Configured implements Tool {
         if (cleanup) cleanRegionRootDir(fs, rootRegionDir);
       }
     } finally {
+      span.end();
       // We may be called inside a test that wants to keep on using the fs.
       if (!noclosefs) {
         fs.close();
       }
-      if (scope != null) {
-        scope.close();
-      }
-      if (receiverHost != null) {
-        receiverHost.closeReceivers();
-      }
     }
 
-    return(0);
+    return 0;
   }
 
   private static TableDescriptor createHTableDescriptor(final int regionNum,
diff --git a/hbase-shaded/hbase-shaded-client/pom.xml b/hbase-shaded/hbase-shaded-client/pom.xml
index 44c8bf6..2d06a9d 100644
--- a/hbase-shaded/hbase-shaded-client/pom.xml
+++ b/hbase-shaded/hbase-shaded-client/pom.xml
@@ -79,6 +79,7 @@
                                     <exclude>log4j:*</exclude>
                                     <exclude>commons-logging:*</exclude>
                                     <exclude>org.javassist:*</exclude>
+                                    <exclude>io.opentelemetry:*</exclude>
                                 </excludes>
                             </artifactSet>
                         </configuration>
diff --git a/hbase-shaded/hbase-shaded-testing-util/pom.xml b/hbase-shaded/hbase-shaded-testing-util/pom.xml
index 6ee31e2..ac6ab49 100644
--- a/hbase-shaded/hbase-shaded-testing-util/pom.xml
+++ b/hbase-shaded/hbase-shaded-testing-util/pom.xml
@@ -236,6 +236,7 @@
                                     <exclude>log4j:*</exclude>
                                     <exclude>commons-logging:*</exclude>
                                     <exclude>org.javassist:*</exclude>
+                                    <exclude>io.opentelemetry:*</exclude>
                                 </excludes>
                             </artifactSet>
                         </configuration>
diff --git a/hbase-shaded/pom.xml b/hbase-shaded/pom.xml
index 2a44f43..d1dfb06 100644
--- a/hbase-shaded/pom.xml
+++ b/hbase-shaded/pom.xml
@@ -159,6 +159,7 @@
                                         <exclude>log4j:*</exclude>
                                         <exclude>commons-logging:*</exclude>
                                         <exclude>org.javassist:*</exclude>
+                                        <exclude>io.opentelemetry:*</exclude>
                                     </excludes>
                                 </artifactSet>
                                 <relocations>
diff --git a/hbase-shell/src/main/ruby/shell/commands/trace.rb b/hbase-shell/src/main/ruby/shell/commands/trace.rb
index f2a8ee0..f52f474 100644
--- a/hbase-shell/src/main/ruby/shell/commands/trace.rb
+++ b/hbase-shell/src/main/ruby/shell/commands/trace.rb
@@ -17,16 +17,17 @@
 # limitations under the License.
 #
 
-java_import org.apache.hadoop.hbase.trace.SpanReceiverHost
+# Disable tracing for now as HTrace does not work any more
+# java_import org.apache.hadoop.hbase.trace.SpanReceiverHost
 
 module Shell
   module Commands
     class Trace < Command
-      @@conf = org.apache.htrace.core.HTraceConfiguration.fromKeyValuePairs(
-        'sampler.classes', 'org.apache.htrace.core.AlwaysSampler'
-      )
-      @@tracer = org.apache.htrace.core.Tracer::Builder.new('HBaseShell').conf(@@conf).build()
-      @@tracescope = nil
+      # @@conf = org.apache.htrace.core.HTraceConfiguration.fromKeyValuePairs(
+      #  'sampler.classes', 'org.apache.htrace.core.AlwaysSampler'
+      # )
+      # @@tracer = org.apache.htrace.core.Tracer::Builder.new('HBaseShell').conf(@@conf).build()
+      # @@tracescope = nil
 
       def help
         <<-EOF
@@ -57,23 +58,23 @@ EOF
       end
 
       def trace(startstop, spanname)
-        @@receiver ||= SpanReceiverHost.getInstance(@shell.hbase.configuration)
-        if startstop == 'start'
-          unless tracing?
-            @@tracescope = @@tracer.newScope(spanname)
-          end
-        elsif startstop == 'stop'
-          if tracing?
-            @@tracescope.close
-            @@tracescope = nil
-          end
-        end
-        tracing?
+        # @@receiver ||= SpanReceiverHost.getInstance(@shell.hbase.configuration)
+        # if startstop == 'start'
+        #   unless tracing?
+        #     @@tracescope = @@tracer.newScope(spanname)
+        #   end
+        # elsif startstop == 'stop'
+        #   if tracing?
+        #    @@tracescope.close
+        #    @@tracescope = nil
+        #   end
+        # end
+        # tracing?
       end
 
-      def tracing?
-        @@tracescope != nil
-      end
+      # def tracing?
+      #   @@tracescope != nil
+      # end
     end
   end
 end
diff --git a/hbase-zookeeper/pom.xml b/hbase-zookeeper/pom.xml
index dd8ab24..1814031 100644
--- a/hbase-zookeeper/pom.xml
+++ b/hbase-zookeeper/pom.xml
@@ -148,6 +148,10 @@
       <groupId>org.apache.zookeeper</groupId>
       <artifactId>zookeeper</artifactId>
     </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-api</artifactId>
+    </dependency>
     <!-- Test dependencies -->
     <dependency>
       <groupId>junit</groupId>
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
index 90cb123..f5cc1b2 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
@@ -18,18 +18,18 @@
  */
 package org.apache.hadoop.hbase.zookeeper;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
-
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.RetryCounter;
 import org.apache.hadoop.hbase.util.RetryCounterFactory;
-import org.apache.htrace.core.TraceScope;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.zookeeper.AsyncCallback;
 import org.apache.zookeeper.CreateMode;
@@ -164,7 +164,8 @@ public class RecoverableZooKeeper {
    * exist.
    */
   public void delete(String path, int version) throws InterruptedException, KeeperException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.delete")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.delete").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       boolean isRetry = false; // False for first attempt, true for all retries.
       while (true) {
@@ -196,6 +197,8 @@ public class RecoverableZooKeeper {
         retryCounter.sleepUntilNextRetry();
         isRetry = true;
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -204,7 +207,8 @@ public class RecoverableZooKeeper {
    * @return A Stat instance
    */
   public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.exists").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
@@ -225,6 +229,8 @@ public class RecoverableZooKeeper {
         }
         retryCounter.sleepUntilNextRetry();
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -233,7 +239,9 @@ public class RecoverableZooKeeper {
    * @return A Stat instance
    */
   public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.exists")) {
+    Span span =
+      TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.exists").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
@@ -255,6 +263,8 @@ public class RecoverableZooKeeper {
         }
         retryCounter.sleepUntilNextRetry();
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -273,7 +283,9 @@ public class RecoverableZooKeeper {
    */
   public List<String> getChildren(String path, Watcher watcher)
     throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
+    Span span =
+      TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.getChildren").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
@@ -294,6 +306,8 @@ public class RecoverableZooKeeper {
         }
         retryCounter.sleepUntilNextRetry();
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -303,7 +317,9 @@ public class RecoverableZooKeeper {
    */
   public List<String> getChildren(String path, boolean watch)
     throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getChildren")) {
+    Span span =
+      TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.getChildren").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
@@ -325,6 +341,8 @@ public class RecoverableZooKeeper {
         }
         retryCounter.sleepUntilNextRetry();
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -334,7 +352,8 @@ public class RecoverableZooKeeper {
    */
   public byte[] getData(String path, Watcher watcher, Stat stat)
     throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.getData").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
@@ -355,6 +374,8 @@ public class RecoverableZooKeeper {
         }
         retryCounter.sleepUntilNextRetry();
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -364,7 +385,9 @@ public class RecoverableZooKeeper {
    */
   public byte[] getData(String path, boolean watch, Stat stat)
     throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getData")) {
+    Span span =
+      TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.getData").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
@@ -386,6 +409,8 @@ public class RecoverableZooKeeper {
         }
         retryCounter.sleepUntilNextRetry();
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -397,7 +422,8 @@ public class RecoverableZooKeeper {
    */
   public Stat setData(String path, byte[] data, int version)
     throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setData")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.setData").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       byte[] newData = ZKMetadata.appendMetaData(id, data);
       boolean isRetry = false;
@@ -437,6 +463,8 @@ public class RecoverableZooKeeper {
         retryCounter.sleepUntilNextRetry();
         isRetry = true;
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -444,9 +472,9 @@ public class RecoverableZooKeeper {
    * getAcl is an idempotent operation. Retry before throwing exception
    * @return list of ACLs
    */
-  public List<ACL> getAcl(String path, Stat stat)
-    throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getAcl")) {
+  public List<ACL> getAcl(String path, Stat stat) throws KeeperException, InterruptedException {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.getAcl").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
@@ -467,6 +495,8 @@ public class RecoverableZooKeeper {
         }
         retryCounter.sleepUntilNextRetry();
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -476,7 +506,8 @@ public class RecoverableZooKeeper {
    */
   public Stat setAcl(String path, List<ACL> acls, int version)
     throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.setAcl")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.setAcl").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       while (true) {
         try {
@@ -496,6 +527,8 @@ public class RecoverableZooKeeper {
         }
         retryCounter.sleepUntilNextRetry();
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -514,10 +547,10 @@ public class RecoverableZooKeeper {
    *
    * @return Path
    */
-  public String create(String path, byte[] data, List<ACL> acl,
-      CreateMode createMode)
+  public String create(String path, byte[] data, List<ACL> acl, CreateMode createMode)
     throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.create")) {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.create").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       byte[] newData = ZKMetadata.appendMetaData(id, data);
       switch (createMode) {
         case EPHEMERAL:
@@ -532,6 +565,8 @@ public class RecoverableZooKeeper {
           throw new IllegalArgumentException("Unrecognized CreateMode: " +
               createMode);
       }
+    } finally {
+      span.end();
     }
   }
 
@@ -647,9 +682,9 @@ public class RecoverableZooKeeper {
   /**
    * Run multiple operations in a transactional manner. Retry before throwing exception
    */
-  public List<OpResult> multi(Iterable<Op> ops)
-    throws KeeperException, InterruptedException {
-    try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.multi")) {
+  public List<OpResult> multi(Iterable<Op> ops) throws KeeperException, InterruptedException {
+    Span span = TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.multi").startSpan();
+    try (Scope scope = span.makeCurrent()) {
       RetryCounter retryCounter = retryCounterFactory.create();
       Iterable<Op> multiOps = prepareZKMulti(ops);
       while (true) {
@@ -671,6 +706,8 @@ public class RecoverableZooKeeper {
         }
         retryCounter.sleepUntilNextRetry();
       }
+    } finally {
+      span.end();
     }
   }
 
diff --git a/pom.xml b/pom.xml
index c6866e6..850fa54 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1027,6 +1027,25 @@
             </configuration>
           </execution>
           <execution>
+            <id>banned-htrace</id>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <rules>
+                <bannedDependencies>
+                  <excludes>
+                    <exclude>org.apache.htrace:**</exclude>
+                  </excludes>
+                  <message>
+                    Use OpenTelemetry instead
+                  </message>
+                  <searchTransitive>false</searchTransitive>
+                </bannedDependencies>
+              </rules>
+            </configuration>
+          </execution>
+          <execution>
             <id>check-aggregate-license</id>
             <!-- must check after LICENSE is built at 'generate-resources' -->
             <phase>process-resources</phase>
@@ -1135,9 +1154,10 @@
                 <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
                   <includeTestCode>true</includeTestCode>
                   <commentLineBufferSize>512</commentLineBufferSize>
-                  <reason>Do not use htrace v3</reason>
+                  <reason>Do not use htrace</reason>
                   <bannedImports>
                     <bannedImport>org.htrace.**</bannedImport>
+                    <bannedImport>org.apache.htrace.**</bannedImport>
                   </bannedImports>
                 </restrictImports>
                 <restrictImports implementation="de.skuzzle.enforcer.restrictimports.rule.RestrictImports">
@@ -1463,7 +1483,7 @@
     <jruby.version>9.2.13.0</jruby.version>
     <junit.version>4.13</junit.version>
     <hamcrest.version>1.3</hamcrest.version>
-    <htrace.version>4.2.0-incubating</htrace.version>
+    <opentelemetry.version>0.12.0</opentelemetry.version>
     <log4j.version>1.2.17</log4j.version>
     <mockito-core.version>2.28.2</mockito-core.version>
     <!--Internally we use a different version of protobuf. See hbase-protocol-shaded-->
@@ -2155,9 +2175,9 @@
         <scope>test</scope>
       </dependency>
       <dependency>
-        <groupId>org.apache.htrace</groupId>
-        <artifactId>htrace-core4</artifactId>
-        <version>${htrace.version}</version>
+        <groupId>io.opentelemetry</groupId>
+        <artifactId>opentelemetry-api</artifactId>
+        <version>${opentelemetry.version}</version>
       </dependency>
       <dependency>
         <groupId>com.lmax</groupId>

[hbase] 04/18: HBASE-26127 Backport HBASE-23898 "Add trace support for simple apis i… (#3556)

Posted by ta...@apache.org.
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 58b5c007b9aa0ea6687aa116ee8605f33496dc3f
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Wed Aug 4 18:25:24 2021 -0700

    HBASE-26127 Backport HBASE-23898 "Add trace support for simple apis i… (#3556)
    
    4/17 commits of HBASE-22120, original commit 805b2ae2ad0f6325515d46043ff01e4e2c7a9f59
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 hbase-client/pom.xml                               |  10 +
 .../hadoop/hbase/client/AsyncConnection.java       |   4 +-
 .../hadoop/hbase/client/AsyncConnectionImpl.java   | 106 +++---
 .../hadoop/hbase/client/AsyncRegionLocator.java    | 166 +++++---
 .../org/apache/hadoop/hbase/client/AsyncTable.java |  31 +-
 .../hbase/client/AsyncTableRegionLocatorImpl.java  |  18 +-
 .../hadoop/hbase/client/ConnectionFactory.java     |  53 +--
 .../hadoop/hbase/client/RawAsyncTableImpl.java     | 345 +++++++++--------
 .../apache/hadoop/hbase/ipc/AbstractRpcClient.java |   9 +-
 .../client/TestAsyncRegionLocatorTracing.java      | 157 ++++++++
 .../hadoop/hbase/client/TestAsyncTableTracing.java | 417 +++++++++++++++++++++
 .../org/apache/hadoop/hbase/trace/TraceUtil.java   | 134 +++++++
 .../org/apache/hadoop/hbase/ipc/CallRunner.java    |   7 +-
 .../hadoop/hbase/ipc/ServerRpcConnection.java      |   3 +-
 .../apache/hadoop/hbase/ipc/AbstractTestIPC.java   |  24 +-
 pom.xml                                            |   3 +-
 16 files changed, 1159 insertions(+), 328 deletions(-)

diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index e62a7d1..01a74cc 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -157,6 +157,16 @@
       <artifactId>joni</artifactId>
     </dependency>
     <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk-testing</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>jcl-over-slf4j</artifactId>
       <scope>test</scope>
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
index 75971ad..d04b5f2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
@@ -64,8 +64,8 @@ public interface AsyncConnection extends Closeable {
   /**
    * Retrieve an {@link AsyncTable} implementation for accessing a table.
    * <p>
-   * The returned instance will use default configs. Use {@link #getTableBuilder(TableName)} if
-   * you want to customize some configs.
+   * The returned instance will use default configs. Use {@link #getTableBuilder(TableName)} if you
+   * want to customize some configs.
    * <p>
    * This method no longer checks table existence. An exception will be thrown if the table does not
    * exist only when the first operation is attempted.
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 2ed7399..b919ee1 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
@@ -27,6 +27,8 @@ import static org.apache.hadoop.hbase.client.MetricsConnection.CLIENT_SIDE_METRI
 import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLED_KEY;
 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;
@@ -48,6 +50,7 @@ import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.ConcurrentMapUtils;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -153,14 +156,13 @@ class AsyncConnectionImpl implements AsyncConnection {
         LOG.warn("{} is true, but {} is not set", STATUS_PUBLISHED, STATUS_LISTENER_CLASS);
       } else {
         try {
-          listener = new ClusterStatusListener(
-            new ClusterStatusListener.DeadServerHandler() {
-              @Override
-              public void newDead(ServerName sn) {
-                locator.clearCache(sn);
-                rpcClient.cancelConnections(sn);
-              }
-            }, conf, listenerClass);
+          listener = new ClusterStatusListener(new ClusterStatusListener.DeadServerHandler() {
+            @Override
+            public void newDead(ServerName sn) {
+              locator.clearCache(sn);
+              rpcClient.cancelConnections(sn);
+            }
+          }, conf, listenerClass);
         } catch (IOException e) {
           LOG.warn("Failed create of ClusterStatusListener, not a critical, ignoring...", e);
         }
@@ -195,27 +197,29 @@ class AsyncConnectionImpl implements AsyncConnection {
 
   @Override
   public void close() {
-    // As the code below is safe to be executed in parallel, here we do not use CAS or lock, just a
-    // simple volatile flag.
-    if (closed) {
-      return;
-    }
-    LOG.info("Connection has been closed by {}.", Thread.currentThread().getName());
-    if(LOG.isDebugEnabled()){
-      logCallStack(Thread.currentThread().getStackTrace());
-    }
-    IOUtils.closeQuietly(clusterStatusListener,
-      e -> LOG.warn("failed to close clusterStatusListener", e));
-    IOUtils.closeQuietly(rpcClient, e -> LOG.warn("failed to close rpcClient", e));
-    IOUtils.closeQuietly(registry, e -> LOG.warn("failed to close registry", e));
-    synchronized (this) {
-      if (choreService != null) {
-        choreService.shutdown();
-        choreService = null;
+    TraceUtil.trace(() -> {
+      // As the code below is safe to be executed in parallel, here we do not use CAS or lock,
+      // just a simple volatile flag.
+      if (closed) {
+        return;
       }
-    }
-    metrics.ifPresent(MetricsConnection::shutdown);
-    closed = true;
+      LOG.info("Connection has been closed by {}.", Thread.currentThread().getName());
+      if (LOG.isDebugEnabled()) {
+        logCallStack(Thread.currentThread().getStackTrace());
+      }
+      IOUtils.closeQuietly(clusterStatusListener,
+        e -> LOG.warn("failed to close clusterStatusListener", e));
+      IOUtils.closeQuietly(rpcClient, e -> LOG.warn("failed to close rpcClient", e));
+      IOUtils.closeQuietly(registry, e -> LOG.warn("failed to close registry", e));
+      synchronized (this) {
+        if (choreService != null) {
+          choreService.shutdown();
+          choreService = null;
+        }
+      }
+      metrics.ifPresent(MetricsConnection::shutdown);
+      closed = true;
+    }, "AsyncConnection.close");
   }
 
   private void logCallStack(StackTraceElement[] stackTraceElements) {
@@ -320,7 +324,7 @@ class AsyncConnectionImpl implements AsyncConnection {
 
   @Override
   public AsyncTableBuilder<ScanResultConsumer> getTableBuilder(TableName tableName,
-      ExecutorService pool) {
+    ExecutorService pool) {
     return new AsyncTableBuilderBase<ScanResultConsumer>(tableName, connConf) {
 
       @Override
@@ -361,35 +365,43 @@ class AsyncConnectionImpl implements AsyncConnection {
 
   @Override
   public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName,
-      ExecutorService pool) {
+    ExecutorService pool) {
     return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool),
       RETRY_TIMER);
   }
 
   @Override
   public CompletableFuture<Hbck> getHbck() {
-    CompletableFuture<Hbck> future = new CompletableFuture<>();
-    addListener(registry.getActiveMaster(), (sn, error) -> {
-      if (error != null) {
-        future.completeExceptionally(error);
-      } else {
-        try {
-          future.complete(getHbck(sn));
-        } catch (IOException e) {
-          future.completeExceptionally(e);
+    return TraceUtil.tracedFuture(() -> {
+      CompletableFuture<Hbck> future = new CompletableFuture<>();
+      addListener(registry.getActiveMaster(), (sn, error) -> {
+        if (error != null) {
+          future.completeExceptionally(error);
+        } else {
+          try {
+            future.complete(getHbck(sn));
+          } catch (IOException e) {
+            future.completeExceptionally(e);
+          }
         }
-      }
-    });
-    return future;
+      });
+      return future;
+    }, getClass().getName() + ".getHbck");
   }
 
   @Override
   public Hbck getHbck(ServerName masterServer) throws IOException {
-    // 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);
+    Span span = TraceUtil.createSpan(getClass().getName() + ".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);
+    }
   }
 
   @Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index d50070a..1d0efcc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -18,16 +18,28 @@
 package org.apache.hadoop.hbase.client;
 
 import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.trace.TraceUtil.REGION_NAMES_KEY;
+import static org.apache.hadoop.hbase.trace.TraceUtil.SERVER_NAME_KEY;
+import static org.apache.hadoop.hbase.trace.TraceUtil.createSpan;
+import static org.apache.hadoop.hbase.trace.TraceUtil.createTableSpan;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.context.Scope;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
 import java.util.function.Supplier;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
+import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
@@ -60,7 +72,7 @@ class AsyncRegionLocator {
   }
 
   private <T> CompletableFuture<T> withTimeout(CompletableFuture<T> future, long timeoutNs,
-      Supplier<String> timeoutMsg) {
+    Supplier<String> timeoutMsg) {
     if (future.isDone() || timeoutNs <= 0) {
       return future;
     }
@@ -83,64 +95,101 @@ class AsyncRegionLocator {
     return TableName.isMetaTableName(tableName);
   }
 
+  private <T> CompletableFuture<T> tracedLocationFuture(Supplier<CompletableFuture<T>> action,
+    Function<T, List<String>> getRegionNames, TableName tableName, String methodName) {
+    Span span = createTableSpan(getClass().getSimpleName() + "." + methodName, tableName);
+    try (Scope scope = span.makeCurrent()) {
+      CompletableFuture<T> future = action.get();
+      FutureUtils.addListener(future, (resp, error) -> {
+        if (error != null) {
+          span.recordException(error);
+          span.setStatus(StatusCode.ERROR);
+        } else {
+          List<String> regionNames = getRegionNames.apply(resp);
+          if (!regionNames.isEmpty()) {
+            span.setAttribute(REGION_NAMES_KEY, regionNames);
+          }
+          span.setStatus(StatusCode.OK);
+        }
+        span.end();
+      });
+      return future;
+    }
+  }
+
+  private List<String> getRegionName(RegionLocations locs) {
+    List<String> names = new ArrayList<>();
+    for (HRegionLocation loc : locs.getRegionLocations()) {
+      if (loc != null) {
+        names.add(loc.getRegion().getRegionNameAsString());
+      }
+    }
+    return names;
+  }
+
   CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
-      RegionLocateType type, boolean reload, long timeoutNs) {
-    CompletableFuture<RegionLocations> future = isMeta(tableName)
-      ? metaRegionLocator.getRegionLocations(RegionReplicaUtil.DEFAULT_REPLICA_ID, reload)
-      : nonMetaRegionLocator.getRegionLocations(tableName, row,
-        RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload);
-    return withTimeout(future, timeoutNs,
-      () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs) +
-        "ms) waiting for region locations for " + tableName + ", row='" +
-        Bytes.toStringBinary(row) + "'");
+    RegionLocateType type, boolean reload, long timeoutNs) {
+    return tracedLocationFuture(() -> {
+      CompletableFuture<RegionLocations> future = isMeta(tableName) ?
+        metaRegionLocator.getRegionLocations(RegionReplicaUtil.DEFAULT_REPLICA_ID, reload) :
+        nonMetaRegionLocator.getRegionLocations(tableName, row,
+          RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload);
+      return withTimeout(future, timeoutNs,
+        () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs) +
+          "ms) waiting for region locations for " + tableName + ", row='" +
+          Bytes.toStringBinary(row) + "'");
+    }, this::getRegionName, tableName, "getRegionLocations");
   }
 
   CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
-      int replicaId, RegionLocateType type, boolean reload, long timeoutNs) {
-    // meta region can not be split right now so we always call the same method.
-    // Change it later if the meta table can have more than one regions.
-    CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
-    CompletableFuture<RegionLocations> locsFuture =
-      isMeta(tableName) ? metaRegionLocator.getRegionLocations(replicaId, reload)
-        : nonMetaRegionLocator.getRegionLocations(tableName, row, replicaId, type, reload);
-    addListener(locsFuture, (locs, error) -> {
-      if (error != null) {
-        future.completeExceptionally(error);
-        return;
-      }
-      HRegionLocation loc = locs.getRegionLocation(replicaId);
-      if (loc == null) {
-        future.completeExceptionally(
-          new RegionOfflineException("No location for " + tableName + ", row='" +
-            Bytes.toStringBinary(row) + "', locateType=" + type + ", replicaId=" + replicaId));
-      } else if (loc.getServerName() == null) {
-        future.completeExceptionally(
-          new RegionOfflineException("No server address listed for region '" +
-            loc.getRegion().getRegionNameAsString() + ", row='" + Bytes.toStringBinary(row) +
-            "', locateType=" + type + ", replicaId=" + replicaId));
-      } else {
-        future.complete(loc);
-      }
-    });
-    return withTimeout(future, timeoutNs,
-      () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs) +
-        "ms) waiting for region location for " + tableName + ", row='" + Bytes.toStringBinary(row) +
-        "', replicaId=" + replicaId);
+    int replicaId, RegionLocateType type, boolean reload, long timeoutNs) {
+    return tracedLocationFuture(() -> {
+      // meta region can not be split right now so we always call the same method.
+      // Change it later if the meta table can have more than one regions.
+      CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
+      CompletableFuture<RegionLocations> locsFuture =
+        isMeta(tableName) ? metaRegionLocator.getRegionLocations(replicaId, reload) :
+          nonMetaRegionLocator.getRegionLocations(tableName, row, replicaId, type, reload);
+      addListener(locsFuture, (locs, error) -> {
+        if (error != null) {
+          future.completeExceptionally(error);
+          return;
+        }
+        HRegionLocation loc = locs.getRegionLocation(replicaId);
+        if (loc == null) {
+          future.completeExceptionally(
+            new RegionOfflineException("No location for " + tableName + ", row='" +
+              Bytes.toStringBinary(row) + "', locateType=" + type + ", replicaId=" + replicaId));
+        } else if (loc.getServerName() == null) {
+          future.completeExceptionally(
+            new RegionOfflineException("No server address listed for region '" +
+              loc.getRegion().getRegionNameAsString() + ", row='" + Bytes.toStringBinary(row) +
+              "', locateType=" + type + ", replicaId=" + replicaId));
+        } else {
+          future.complete(loc);
+        }
+      });
+      return withTimeout(future, timeoutNs,
+        () -> "Timeout(" + TimeUnit.NANOSECONDS.toMillis(timeoutNs) +
+          "ms) waiting for region location for " + tableName + ", row='" +
+          Bytes.toStringBinary(row) + "', replicaId=" + replicaId);
+    }, loc -> Arrays.asList(loc.getRegion().getRegionNameAsString()), tableName,
+      "getRegionLocation");
   }
 
   CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
-      int replicaId, RegionLocateType type, long timeoutNs) {
+    int replicaId, RegionLocateType type, long timeoutNs) {
     return getRegionLocation(tableName, row, replicaId, type, false, timeoutNs);
   }
 
   CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
-      RegionLocateType type, boolean reload, long timeoutNs) {
+    RegionLocateType type, boolean reload, long timeoutNs) {
     return getRegionLocation(tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload,
       timeoutNs);
   }
 
   CompletableFuture<HRegionLocation> getRegionLocation(TableName tableName, byte[] row,
-      RegionLocateType type, long timeoutNs) {
+    RegionLocateType type, long timeoutNs) {
     return getRegionLocation(tableName, row, type, false, timeoutNs);
   }
 
@@ -153,24 +202,31 @@ class AsyncRegionLocator {
   }
 
   void clearCache(TableName tableName) {
-    LOG.debug("Clear meta cache for {}", tableName);
-    if (tableName.equals(META_TABLE_NAME)) {
-      metaRegionLocator.clearCache();
-    } else {
-      nonMetaRegionLocator.clearCache(tableName);
-    }
+    TraceUtil.trace(() -> {
+      LOG.debug("Clear meta cache for {}", tableName);
+      if (tableName.equals(META_TABLE_NAME)) {
+        metaRegionLocator.clearCache();
+      } else {
+        nonMetaRegionLocator.clearCache(tableName);
+      }
+    }, () -> createTableSpan("AsyncRegionLocator.clearCache", tableName));
   }
 
   void clearCache(ServerName serverName) {
-    LOG.debug("Clear meta cache for {}", serverName);
-    metaRegionLocator.clearCache(serverName);
-    nonMetaRegionLocator.clearCache(serverName);
-    conn.getConnectionMetrics().ifPresent(MetricsConnection::incrMetaCacheNumClearServer);
+    TraceUtil.trace(() -> {
+      LOG.debug("Clear meta cache for {}", serverName);
+      metaRegionLocator.clearCache(serverName);
+      nonMetaRegionLocator.clearCache(serverName);
+      conn.getConnectionMetrics().ifPresent(MetricsConnection::incrMetaCacheNumClearServer);
+    }, () -> createSpan("AsyncRegionLocator.clearCache").setAttribute(SERVER_NAME_KEY,
+      serverName.getServerName()));
   }
 
   void clearCache() {
-    metaRegionLocator.clearCache();
-    nonMetaRegionLocator.clearCache();
+    TraceUtil.trace(() -> {
+      metaRegionLocator.clearCache();
+      nonMetaRegionLocator.clearCache();
+    }, "AsyncRegionLocator.clearCache");
   }
 
   AsyncNonMetaRegionLocator getNonMetaRegionLocator() {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java
index 7473ed0..c7003e0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java
@@ -70,6 +70,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
    * Gets the {@link AsyncTableRegionLocator} for this table.
    */
   AsyncTableRegionLocator getRegionLocator();
+
   /**
    * Get timeout of each rpc request in this Table instance. It will be overridden by a more
    * specific rpc timeout config such as readRpcTimeout or writeRpcTimeout.
@@ -184,7 +185,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
    *         {@link CompletableFuture}.
    */
   default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
-      long amount) {
+    long amount) {
     return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
   }
 
@@ -204,12 +205,12 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
    *         {@link CompletableFuture}.
    */
   default CompletableFuture<Long> incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
-      long amount, Durability durability) {
+    long amount, Durability durability) {
     Preconditions.checkNotNull(row, "row is null");
     Preconditions.checkNotNull(family, "family is null");
     return increment(
       new Increment(row).addColumn(family, qualifier, amount).setDurability(durability))
-          .thenApply(r -> Bytes.toLong(r.getValue(family, qualifier)));
+        .thenApply(r -> Bytes.toLong(r.getValue(family, qualifier)));
   }
 
   /**
@@ -361,9 +362,8 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
   }
 
   /**
-   * checkAndMutate that atomically checks if a row matches the specified condition. If it does,
-   * it performs the specified action.
-   *
+   * checkAndMutate that atomically checks if a row matches the specified condition. If it does, it
+   * performs the specified action.
    * @param checkAndMutate The CheckAndMutate object.
    * @return A {@link CompletableFuture}s that represent the result for the CheckAndMutate.
    */
@@ -373,22 +373,19 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
    * Batch version of checkAndMutate. The specified CheckAndMutates are batched only in the sense
    * that they are sent to a RS in one RPC, but each CheckAndMutate operation is still executed
    * atomically (and thus, each may fail independently of others).
-   *
    * @param checkAndMutates The list of CheckAndMutate.
-   * @return A list of {@link CompletableFuture}s that represent the result for each
-   *   CheckAndMutate.
+   * @return A list of {@link CompletableFuture}s that represent the result for each CheckAndMutate.
    */
-  List<CompletableFuture<CheckAndMutateResult>> checkAndMutate(
-    List<CheckAndMutate> checkAndMutates);
+  List<CompletableFuture<CheckAndMutateResult>>
+    checkAndMutate(List<CheckAndMutate> checkAndMutates);
 
   /**
    * A simple version of batch checkAndMutate. It will fail if there are any failures.
-   *
    * @param checkAndMutates The list of rows to apply.
    * @return A {@link CompletableFuture} that wrapper the result list.
    */
-  default CompletableFuture<List<CheckAndMutateResult>> checkAndMutateAll(
-    List<CheckAndMutate> checkAndMutates) {
+  default CompletableFuture<List<CheckAndMutateResult>>
+    checkAndMutateAll(List<CheckAndMutate> checkAndMutates) {
     return allOf(checkAndMutate(checkAndMutates));
   }
 
@@ -484,7 +481,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
    */
   default List<CompletableFuture<Boolean>> exists(List<Get> gets) {
     return get(toCheckExistenceOnly(gets)).stream()
-        .<CompletableFuture<Boolean>> map(f -> f.thenApply(r -> r.getExists())).collect(toList());
+      .<CompletableFuture<Boolean>> map(f -> f.thenApply(r -> r.getExists())).collect(toList());
   }
 
   /**
@@ -592,7 +589,7 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
    * @see ServiceCaller
    */
   <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
-      ServiceCaller<S, R> callable, byte[] row);
+    ServiceCaller<S, R> callable, byte[] row);
 
   /**
    * The callback when we want to execute a coprocessor call on a range of regions.
@@ -731,5 +728,5 @@ public interface AsyncTable<C extends ScanResultConsumerBase> {
    *          for more details.
    */
   <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(Function<RpcChannel, S> stubMaker,
-      ServiceCaller<S, R> callable, CoprocessorCallback<R> callback);
+    ServiceCaller<S, R> callable, CoprocessorCallback<R> callback);
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
index fa3ea1c..d5b275d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hbase.client;
 
+import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFuture;
+
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
@@ -47,19 +49,21 @@ class AsyncTableRegionLocatorImpl implements AsyncTableRegionLocator {
 
   @Override
   public CompletableFuture<HRegionLocation> getRegionLocation(byte[] row, int replicaId,
-      boolean reload) {
+    boolean reload) {
     return conn.getLocator().getRegionLocation(tableName, row, replicaId, RegionLocateType.CURRENT,
       reload, -1L);
   }
 
   @Override
   public CompletableFuture<List<HRegionLocation>> getAllRegionLocations() {
-    if (TableName.isMetaTableName(tableName)) {
-      return conn.registry.getMetaRegionLocations()
-        .thenApply(locs -> Arrays.asList(locs.getRegionLocations()));
-    }
-    return AsyncMetaTableAccessor.getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME),
-      tableName);
+    return tracedFuture(() -> {
+      if (TableName.isMetaTableName(tableName)) {
+        return conn.registry.getMetaRegionLocations()
+          .thenApply(locs -> Arrays.asList(locs.getRegionLocations()));
+      }
+      return AsyncMetaTableAccessor
+        .getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME), tableName);
+    }, getClass().getSimpleName() + ".getAllRegionLocations");
   }
 
   @Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
index f91b210..627e8d2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.ReflectionUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 
@@ -280,30 +281,32 @@ public class ConnectionFactory {
    */
   public static CompletableFuture<AsyncConnection> createAsyncConnection(Configuration conf,
       final User user) {
-    CompletableFuture<AsyncConnection> future = new CompletableFuture<>();
-    ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf);
-    addListener(registry.getClusterId(), (clusterId, error) -> {
-      if (error != null) {
-        registry.close();
-        future.completeExceptionally(error);
-        return;
-      }
-      if (clusterId == null) {
-        registry.close();
-        future.completeExceptionally(new IOException("clusterid came back null"));
-        return;
-      }
-      Class<? extends AsyncConnection> clazz = conf.getClass(HBASE_CLIENT_ASYNC_CONNECTION_IMPL,
-        AsyncConnectionImpl.class, AsyncConnection.class);
-      try {
-        future.complete(
-          user.runAs((PrivilegedExceptionAction<? extends AsyncConnection>) () -> ReflectionUtils
-            .newInstance(clazz, conf, registry, clusterId, user)));
-      } catch (Exception e) {
-        registry.close();
-        future.completeExceptionally(e);
-      }
-    });
-    return future;
+    return TraceUtil.tracedFuture(() -> {
+      CompletableFuture<AsyncConnection> future = new CompletableFuture<>();
+      ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf);
+      addListener(registry.getClusterId(), (clusterId, error) -> {
+        if (error != null) {
+          registry.close();
+          future.completeExceptionally(error);
+          return;
+        }
+        if (clusterId == null) {
+          registry.close();
+          future.completeExceptionally(new IOException("clusterid came back null"));
+          return;
+        }
+        Class<? extends AsyncConnection> clazz = conf.getClass(HBASE_CLIENT_ASYNC_CONNECTION_IMPL,
+          AsyncConnectionImpl.class, AsyncConnection.class);
+        try {
+          future.complete(user.runAs(
+            (PrivilegedExceptionAction<? extends AsyncConnection>) () -> ReflectionUtils
+              .newInstance(clazz, conf, registry, clusterId, user)));
+        } catch (Exception e) {
+          registry.close();
+          future.completeExceptionally(e);
+        }
+      });
+      return future;
+    }, ConnectionFactory.class.getSimpleName() + ".createAsyncConnection");
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
index bed896e..f637e47 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncTableImpl.java
@@ -23,6 +23,8 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.timelineConsistentRead;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePut;
 import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePutsInRowMutations;
+import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFuture;
+import static org.apache.hadoop.hbase.trace.TraceUtil.tracedFutures;
 import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 
 import com.google.protobuf.RpcChannel;
@@ -131,8 +133,8 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
     }
     this.maxAttempts = builder.maxAttempts;
     this.startLogErrorsCnt = builder.startLogErrorsCnt;
-    this.defaultScannerCaching = tableName.isSystemTable() ? conn.connConf.getMetaScannerCaching()
-      : conn.connConf.getScannerCaching();
+    this.defaultScannerCaching = tableName.isSystemTable() ? conn.connConf.getMetaScannerCaching() :
+      conn.connConf.getScannerCaching();
     this.defaultScannerMaxResultSize = conn.connConf.getScannerMaxResultSize();
   }
 
@@ -204,15 +206,15 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
   }
 
   private static <REQ> CompletableFuture<Void> voidMutate(HBaseRpcController controller,
-      HRegionLocation loc, ClientService.Interface stub, REQ req,
-      Converter<MutateRequest, byte[], REQ> reqConvert) {
+    HRegionLocation loc, ClientService.Interface stub, REQ req,
+    Converter<MutateRequest, byte[], REQ> reqConvert) {
     return mutate(controller, loc, stub, req, reqConvert, (c, resp) -> {
       return null;
     });
   }
 
   private static Result toResult(HBaseRpcController controller, MutateResponse resp)
-      throws IOException {
+    throws IOException {
     if (!resp.hasResult()) {
       return null;
     }
@@ -225,9 +227,9 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
   }
 
   private <REQ, RESP> CompletableFuture<RESP> noncedMutate(long nonceGroup, long nonce,
-      HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub, REQ req,
-      NoncedConverter<MutateRequest, byte[], REQ> reqConvert,
-      Converter<RESP, HBaseRpcController, MutateResponse> respConverter) {
+    HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub, REQ req,
+    NoncedConverter<MutateRequest, byte[], REQ> reqConvert,
+    Converter<RESP, HBaseRpcController, MutateResponse> respConverter) {
     return mutate(controller, loc, stub, req,
       (info, src) -> reqConvert.convert(info, src, nonceGroup, nonce), respConverter);
   }
@@ -240,8 +242,8 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
       .maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt);
   }
 
-  private <T, R extends OperationWithAttributes & Row> SingleRequestCallerBuilder<T> newCaller(
-      R row, long rpcTimeoutNs) {
+  private <T, R extends OperationWithAttributes & Row> SingleRequestCallerBuilder<T>
+    newCaller(R row, long rpcTimeoutNs) {
     return newCaller(row.getRow(), row.getPriority(), rpcTimeoutNs);
   }
 
@@ -256,50 +258,58 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
   @Override
   public CompletableFuture<Result> get(Get get) {
-    return timelineConsistentRead(conn.getLocator(), tableName, get, get.getRow(),
-      RegionLocateType.CURRENT, replicaId -> get(get, replicaId), readRpcTimeoutNs,
-      conn.connConf.getPrimaryCallTimeoutNs(), retryTimer, conn.getConnectionMetrics());
+    return tracedFuture(
+      () -> timelineConsistentRead(conn.getLocator(), tableName, get, get.getRow(),
+        RegionLocateType.CURRENT, replicaId -> get(get, replicaId), readRpcTimeoutNs,
+        conn.connConf.getPrimaryCallTimeoutNs(), retryTimer, conn.getConnectionMetrics()),
+      "AsyncTable.get", tableName);
   }
 
   @Override
   public CompletableFuture<Void> put(Put put) {
     validatePut(put, conn.connConf.getMaxKeyValueSize());
-    return this.<Void, Put> newCaller(put, writeRpcTimeoutNs)
+    return tracedFuture(() -> this.<Void, Put> newCaller(put, writeRpcTimeoutNs)
       .action((controller, loc, stub) -> RawAsyncTableImpl.<Put> voidMutate(controller, loc, stub,
         put, RequestConverter::buildMutateRequest))
-      .call();
+      .call(), "AsyncTable.put", tableName);
   }
 
   @Override
   public CompletableFuture<Void> delete(Delete delete) {
-    return this.<Void, Delete> newCaller(delete, writeRpcTimeoutNs)
-      .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete> voidMutate(controller, loc,
-        stub, delete, RequestConverter::buildMutateRequest))
-      .call();
+    return tracedFuture(
+      () -> this.<Void, Delete> newCaller(delete, writeRpcTimeoutNs)
+        .action((controller, loc, stub) -> RawAsyncTableImpl.<Delete> voidMutate(controller, loc,
+          stub, delete, RequestConverter::buildMutateRequest))
+        .call(),
+      "AsyncTable.delete", tableName);
   }
 
   @Override
   public CompletableFuture<Result> append(Append append) {
     checkHasFamilies(append);
-    long nonceGroup = conn.getNonceGenerator().getNonceGroup();
-    long nonce = conn.getNonceGenerator().newNonce();
-    return this.<Result, Append> newCaller(append, rpcTimeoutNs)
-      .action(
-        (controller, loc, stub) -> this.<Append, Result> noncedMutate(nonceGroup, nonce, controller,
-          loc, stub, append, RequestConverter::buildMutateRequest, RawAsyncTableImpl::toResult))
-      .call();
+    return tracedFuture(() -> {
+      long nonceGroup = conn.getNonceGenerator().getNonceGroup();
+      long nonce = conn.getNonceGenerator().newNonce();
+      return this.<Result, Append> newCaller(append, rpcTimeoutNs)
+        .action((controller, loc, stub) -> this.<Append, Result> noncedMutate(nonceGroup, nonce,
+          controller, loc, stub, append, RequestConverter::buildMutateRequest,
+          RawAsyncTableImpl::toResult))
+        .call();
+    }, "AsyncTable.append", tableName);
   }
 
   @Override
   public CompletableFuture<Result> increment(Increment increment) {
     checkHasFamilies(increment);
-    long nonceGroup = conn.getNonceGenerator().getNonceGroup();
-    long nonce = conn.getNonceGenerator().newNonce();
-    return this.<Result, Increment> newCaller(increment, rpcTimeoutNs)
-      .action((controller, loc, stub) -> this.<Increment, Result> noncedMutate(nonceGroup, nonce,
-        controller, loc, stub, increment, RequestConverter::buildMutateRequest,
-        RawAsyncTableImpl::toResult))
-      .call();
+    return tracedFuture(() -> {
+      long nonceGroup = conn.getNonceGenerator().getNonceGroup();
+      long nonce = conn.getNonceGenerator().newNonce();
+      return this.<Result, Increment> newCaller(increment, rpcTimeoutNs)
+        .action((controller, loc, stub) -> this.<Increment, Result> noncedMutate(nonceGroup, nonce,
+          controller, loc, stub, increment, RequestConverter::buildMutateRequest,
+          RawAsyncTableImpl::toResult))
+        .call();
+    }, "AsyncTable.increment", tableName);
   }
 
   private final class CheckAndMutateBuilderImpl implements CheckAndMutateBuilder {
@@ -357,39 +367,43 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
     public CompletableFuture<Boolean> thenPut(Put put) {
       validatePut(put, conn.connConf.getMaxKeyValueSize());
       preCheck();
-      return RawAsyncTableImpl.this.<Boolean> newCaller(row, put.getPriority(), rpcTimeoutNs)
-        .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc,
-          stub, put,
-          (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,
-            null, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE),
-          (c, r) -> r.getProcessed()))
-        .call();
+      return tracedFuture(
+        () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, put.getPriority(), rpcTimeoutNs)
+          .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, put,
+            (rn, p) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,
+              null, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE),
+            (c, r) -> r.getProcessed()))
+          .call(),
+        "AsyncTable.CheckAndMutateBuilder.thenPut", tableName);
     }
 
     @Override
     public CompletableFuture<Boolean> thenDelete(Delete delete) {
       preCheck();
-      return RawAsyncTableImpl.this.<Boolean> newCaller(row, delete.getPriority(), rpcTimeoutNs)
-        .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller,
-          loc, stub, delete,
-          (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,
-            null, timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE),
-          (c, r) -> r.getProcessed()))
-        .call();
+      return tracedFuture(
+        () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, delete.getPriority(), rpcTimeoutNs)
+          .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, delete,
+            (rn, d) -> RequestConverter.buildMutateRequest(rn, row, family, qualifier, op, value,
+              null, timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE),
+            (c, r) -> r.getProcessed()))
+          .call(),
+        "AsyncTable.CheckAndMutateBuilder.thenDelete", tableName);
     }
 
     @Override
     public CompletableFuture<Boolean> thenMutate(RowMutations mutation) {
       preCheck();
       validatePutsInRowMutations(mutation, conn.connConf.getMaxKeyValueSize());
-      return RawAsyncTableImpl.this.<Boolean> newCaller(row, mutation.getMaxPriority(),
-        rpcTimeoutNs)
-        .action((controller, loc, stub) -> RawAsyncTableImpl.this.mutateRow(controller,
-          loc, stub, mutation,
-          (rn, rm) -> RequestConverter.buildMultiRequest(rn, row, family, qualifier, op, value,
-            null, timeRange, rm, HConstants.NO_NONCE, HConstants.NO_NONCE),
-          CheckAndMutateResult::isSuccess))
-        .call();
+      return tracedFuture(
+        () -> RawAsyncTableImpl.this
+          .<Boolean> newCaller(row, mutation.getMaxPriority(), rpcTimeoutNs)
+          .action((controller, loc, stub) -> RawAsyncTableImpl.this.mutateRow(controller, loc, stub,
+            mutation,
+            (rn, rm) -> RequestConverter.buildMultiRequest(rn, row, family, qualifier, op, value,
+              null, timeRange, rm, HConstants.NO_NONCE, HConstants.NO_NONCE),
+            CheckAndMutateResult::isSuccess))
+          .call(),
+        "AsyncTable.CheckAndMutateBuilder.thenMutate", tableName);
     }
   }
 
@@ -421,37 +435,42 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
     @Override
     public CompletableFuture<Boolean> thenPut(Put put) {
       validatePut(put, conn.connConf.getMaxKeyValueSize());
-      return RawAsyncTableImpl.this.<Boolean> newCaller(row, put.getPriority(), rpcTimeoutNs)
+      return tracedFuture(
+        () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, put.getPriority(), rpcTimeoutNs)
         .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc,
           stub, put,
           (rn, p) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null,
             filter, timeRange, p, HConstants.NO_NONCE, HConstants.NO_NONCE),
           (c, r) -> r.getProcessed()))
-        .call();
+        .call(),
+        "AsyncTable.CheckAndMutateWithFilterBuilder.thenPut", tableName);
     }
 
     @Override
     public CompletableFuture<Boolean> thenDelete(Delete delete) {
-      return RawAsyncTableImpl.this.<Boolean> newCaller(row, delete.getPriority(), rpcTimeoutNs)
-        .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller,
-          loc, stub, delete,
-          (rn, d) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null,
-            filter, timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE),
-          (c, r) -> r.getProcessed()))
-        .call();
+      return tracedFuture(
+        () -> RawAsyncTableImpl.this.<Boolean> newCaller(row, delete.getPriority(), rpcTimeoutNs)
+          .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, delete,
+            (rn, d) -> RequestConverter.buildMutateRequest(rn, row, null, null, null, null, filter,
+              timeRange, d, HConstants.NO_NONCE, HConstants.NO_NONCE),
+            (c, r) -> r.getProcessed()))
+          .call(),
+        "AsyncTable.CheckAndMutateWithFilterBuilder.thenDelete", tableName);
     }
 
     @Override
     public CompletableFuture<Boolean> thenMutate(RowMutations mutation) {
       validatePutsInRowMutations(mutation, conn.connConf.getMaxKeyValueSize());
-      return RawAsyncTableImpl.this.<Boolean> newCaller(row, mutation.getMaxPriority(),
-        rpcTimeoutNs)
-        .action((controller, loc, stub) -> RawAsyncTableImpl.this.mutateRow(controller,
-          loc, stub, mutation,
-          (rn, rm) -> RequestConverter.buildMultiRequest(rn, row, null, null, null, null,
-            filter, timeRange, rm, HConstants.NO_NONCE, HConstants.NO_NONCE),
-          CheckAndMutateResult::isSuccess))
-        .call();
+      return tracedFuture(
+        () -> RawAsyncTableImpl.this
+          .<Boolean> newCaller(row, mutation.getMaxPriority(), rpcTimeoutNs)
+          .action((controller, loc, stub) -> RawAsyncTableImpl.this.mutateRow(controller, loc, stub,
+            mutation,
+            (rn, rm) -> RequestConverter.buildMultiRequest(rn, row, null, null, null, null, filter,
+              timeRange, rm, HConstants.NO_NONCE, HConstants.NO_NONCE),
+            CheckAndMutateResult::isSuccess))
+          .call(),
+        "AsyncTable.CheckAndMutateWithFilterBuilder.thenMutate", tableName);
     }
   }
 
@@ -462,63 +481,69 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
   @Override
   public CompletableFuture<CheckAndMutateResult> checkAndMutate(CheckAndMutate checkAndMutate) {
-    if (checkAndMutate.getAction() instanceof Put || checkAndMutate.getAction() instanceof Delete
-      || checkAndMutate.getAction() instanceof Increment
-      || checkAndMutate.getAction() instanceof Append) {
-      Mutation mutation = (Mutation) checkAndMutate.getAction();
-      if (mutation instanceof Put) {
-        validatePut((Put) mutation, conn.connConf.getMaxKeyValueSize());
+    return tracedFuture(() -> {
+      if (checkAndMutate.getAction() instanceof Put ||
+        checkAndMutate.getAction() instanceof Delete ||
+        checkAndMutate.getAction() instanceof Increment ||
+        checkAndMutate.getAction() instanceof Append) {
+        Mutation mutation = (Mutation) checkAndMutate.getAction();
+        if (mutation instanceof Put) {
+          validatePut((Put) mutation, conn.connConf.getMaxKeyValueSize());
+        }
+        long nonceGroup = conn.getNonceGenerator().getNonceGroup();
+        long nonce = conn.getNonceGenerator().newNonce();
+        return RawAsyncTableImpl.this
+          .<CheckAndMutateResult> newCaller(checkAndMutate.getRow(), mutation.getPriority(),
+            rpcTimeoutNs)
+          .action(
+            (controller, loc, stub) -> RawAsyncTableImpl.mutate(controller, loc, stub, mutation,
+              (rn, m) -> RequestConverter.buildMutateRequest(rn, checkAndMutate.getRow(),
+                checkAndMutate.getFamily(), checkAndMutate.getQualifier(),
+                checkAndMutate.getCompareOp(), checkAndMutate.getValue(),
+                checkAndMutate.getFilter(), checkAndMutate.getTimeRange(), m, nonceGroup, nonce),
+              (c, r) -> ResponseConverter.getCheckAndMutateResult(r, c.cellScanner())))
+          .call();
+      } else if (checkAndMutate.getAction() instanceof RowMutations) {
+        RowMutations rowMutations = (RowMutations) checkAndMutate.getAction();
+        validatePutsInRowMutations(rowMutations, conn.connConf.getMaxKeyValueSize());
+        long nonceGroup = conn.getNonceGenerator().getNonceGroup();
+        long nonce = conn.getNonceGenerator().newNonce();
+        return RawAsyncTableImpl.this
+          .<CheckAndMutateResult> newCaller(checkAndMutate.getRow(), rowMutations.getMaxPriority(),
+            rpcTimeoutNs)
+          .action((controller, loc, stub) -> RawAsyncTableImpl.this
+            .<CheckAndMutateResult, CheckAndMutateResult> mutateRow(controller, loc, stub,
+              rowMutations,
+              (rn, rm) -> RequestConverter.buildMultiRequest(rn, checkAndMutate.getRow(),
+                checkAndMutate.getFamily(), checkAndMutate.getQualifier(),
+                checkAndMutate.getCompareOp(), checkAndMutate.getValue(),
+                checkAndMutate.getFilter(), checkAndMutate.getTimeRange(), rm, nonceGroup, nonce),
+              resp -> resp))
+          .call();
+      } else {
+        CompletableFuture<CheckAndMutateResult> future = new CompletableFuture<>();
+        future.completeExceptionally(new DoNotRetryIOException(
+          "CheckAndMutate doesn't support " + checkAndMutate.getAction().getClass().getName()));
+        return future;
       }
-      long nonceGroup = conn.getNonceGenerator().getNonceGroup();
-      long nonce = conn.getNonceGenerator().newNonce();
-      return RawAsyncTableImpl.this.<CheckAndMutateResult> newCaller(checkAndMutate.getRow(),
-        mutation.getPriority(), rpcTimeoutNs)
-        .action((controller, loc, stub) -> RawAsyncTableImpl.mutate(controller,
-          loc, stub, mutation,
-          (rn, m) -> RequestConverter.buildMutateRequest(rn, checkAndMutate.getRow(),
-            checkAndMutate.getFamily(), checkAndMutate.getQualifier(),
-            checkAndMutate.getCompareOp(), checkAndMutate.getValue(), checkAndMutate.getFilter(),
-            checkAndMutate.getTimeRange(), m, nonceGroup, nonce),
-          (c, r) -> ResponseConverter.getCheckAndMutateResult(r, c.cellScanner())))
-        .call();
-    } else if (checkAndMutate.getAction() instanceof RowMutations) {
-      RowMutations rowMutations = (RowMutations) checkAndMutate.getAction();
-      validatePutsInRowMutations(rowMutations, conn.connConf.getMaxKeyValueSize());
-      long nonceGroup = conn.getNonceGenerator().getNonceGroup();
-      long nonce = conn.getNonceGenerator().newNonce();
-      return RawAsyncTableImpl.this.<CheckAndMutateResult> newCaller(checkAndMutate.getRow(),
-        rowMutations.getMaxPriority(), rpcTimeoutNs)
-        .action((controller, loc, stub) ->
-          RawAsyncTableImpl.this.<CheckAndMutateResult, CheckAndMutateResult> mutateRow(
-            controller, loc, stub, rowMutations,
-            (rn, rm) -> RequestConverter.buildMultiRequest(rn, checkAndMutate.getRow(),
-            checkAndMutate.getFamily(), checkAndMutate.getQualifier(),
-            checkAndMutate.getCompareOp(), checkAndMutate.getValue(), checkAndMutate.getFilter(),
-            checkAndMutate.getTimeRange(), rm, nonceGroup, nonce),
-            resp -> resp))
-        .call();
-    } else {
-      CompletableFuture<CheckAndMutateResult> future = new CompletableFuture<>();
-      future.completeExceptionally(new DoNotRetryIOException(
-        "CheckAndMutate doesn't support " + checkAndMutate.getAction().getClass().getName()));
-      return future;
-    }
+    }, "AsyncTable.checkAndMutate", tableName);
   }
 
   @Override
-  public List<CompletableFuture<CheckAndMutateResult>> checkAndMutate(
-    List<CheckAndMutate> checkAndMutates) {
-    return batch(checkAndMutates, rpcTimeoutNs).stream()
-      .map(f -> f.thenApply(r -> (CheckAndMutateResult) r)).collect(toList());
+  public List<CompletableFuture<CheckAndMutateResult>>
+    checkAndMutate(List<CheckAndMutate> checkAndMutates) {
+    return tracedFutures(
+      () -> batch(checkAndMutates, rpcTimeoutNs).stream()
+        .map(f -> f.thenApply(r -> (CheckAndMutateResult) r)).collect(toList()),
+      "AsyncTable.checkAndMutateList", tableName);
   }
 
   // We need the MultiRequest when constructing the org.apache.hadoop.hbase.client.MultiResponse,
   // so here I write a new method as I do not want to change the abstraction of call method.
   @SuppressWarnings("unchecked")
   private <RES, RESP> CompletableFuture<RESP> mutateRow(HBaseRpcController controller,
-      HRegionLocation loc, ClientService.Interface stub, RowMutations mutation,
-      Converter<MultiRequest, byte[], RowMutations> reqConvert,
-      Function<RES, RESP> respConverter) {
+    HRegionLocation loc, ClientService.Interface stub, RowMutations mutation,
+    Converter<MultiRequest, byte[], RowMutations> reqConvert, Function<RES, RESP> respConverter) {
     CompletableFuture<RESP> future = new CompletableFuture<>();
     try {
       byte[] regionName = loc.getRegion().getRegionName();
@@ -537,12 +562,12 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
                 loc.getServerName(), multiResp);
               Throwable ex = multiResp.getException(regionName);
               if (ex != null) {
-                future.completeExceptionally(ex instanceof IOException ? ex
-                  : new IOException(
+                future.completeExceptionally(ex instanceof IOException ? ex :
+                  new IOException(
                     "Failed to mutate row: " + Bytes.toStringBinary(mutation.getRow()), ex));
               } else {
-                future.complete(respConverter
-                  .apply((RES) multiResp.getResults().get(regionName).result.get(0)));
+                future.complete(
+                  respConverter.apply((RES) multiResp.getResults().get(regionName).result.get(0)));
               }
             } catch (IOException e) {
               future.completeExceptionally(e);
@@ -561,12 +586,14 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
     validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
     long nonceGroup = conn.getNonceGenerator().getNonceGroup();
     long nonce = conn.getNonceGenerator().newNonce();
-    return this.<Result> newCaller(mutations.getRow(), mutations.getMaxPriority(),
-      writeRpcTimeoutNs).action((controller, loc, stub) ->
-        this.<Result, Result> mutateRow(controller, loc, stub, mutations,
-          (rn, rm) -> RequestConverter.buildMultiRequest(rn, rm, nonceGroup, nonce),
+    return tracedFuture(
+      () -> this
+        .<Result> newCaller(mutations.getRow(), mutations.getMaxPriority(), writeRpcTimeoutNs)
+        .action((controller, loc, stub) -> this.<Result, Result> mutateRow(controller, loc, stub,
+          mutations, (rn, rm) -> RequestConverter.buildMultiRequest(rn, rm, nonceGroup, nonce),
           resp -> resp))
-      .call();
+        .call(),
+      "AsyncTable.mutateRow", tableName);
   }
 
   private Scan setDefaultScanConfig(Scan scan) {
@@ -602,46 +629,48 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
   @Override
   public CompletableFuture<List<Result>> scanAll(Scan scan) {
-    CompletableFuture<List<Result>> future = new CompletableFuture<>();
-    List<Result> scanResults = new ArrayList<>();
-    scan(scan, new AdvancedScanResultConsumer() {
+    return tracedFuture(() -> {
+      CompletableFuture<List<Result>> future = new CompletableFuture<>();
+      List<Result> scanResults = new ArrayList<>();
+      scan(scan, new AdvancedScanResultConsumer() {
 
-      @Override
-      public void onNext(Result[] results, ScanController controller) {
-        scanResults.addAll(Arrays.asList(results));
-      }
+        @Override
+        public void onNext(Result[] results, ScanController controller) {
+          scanResults.addAll(Arrays.asList(results));
+        }
 
-      @Override
-      public void onError(Throwable error) {
-        future.completeExceptionally(error);
-      }
+        @Override
+        public void onError(Throwable error) {
+          future.completeExceptionally(error);
+        }
 
-      @Override
-      public void onComplete() {
-        future.complete(scanResults);
-      }
-    });
-    return future;
+        @Override
+        public void onComplete() {
+          future.complete(scanResults);
+        }
+      });
+      return future;
+    }, "AsyncTable.scanAll", tableName);
   }
 
   @Override
   public List<CompletableFuture<Result>> get(List<Get> gets) {
-    return batch(gets, readRpcTimeoutNs);
+    return tracedFutures(() -> batch(gets, readRpcTimeoutNs), "AsyncTable.getList", tableName);
   }
 
   @Override
   public List<CompletableFuture<Void>> put(List<Put> puts) {
-    return voidMutate(puts);
+    return tracedFutures(() -> voidMutate(puts), "AsyncTable.putList", tableName);
   }
 
   @Override
   public List<CompletableFuture<Void>> delete(List<Delete> deletes) {
-    return voidMutate(deletes);
+    return tracedFutures(() -> voidMutate(deletes), "AsyncTable.deleteList", tableName);
   }
 
   @Override
   public <T> List<CompletableFuture<T>> batch(List<? extends Row> actions) {
-    return batch(actions, rpcTimeoutNs);
+    return tracedFutures(() -> batch(actions, rpcTimeoutNs), "AsyncTable.batch", tableName);
   }
 
   private List<CompletableFuture<Void>> voidMutate(List<? extends Row> actions) {
@@ -698,7 +727,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
   }
 
   private <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
-      ServiceCaller<S, R> callable, RegionInfo region, byte[] row) {
+    ServiceCaller<S, R> callable, RegionInfo region, byte[] row) {
     RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
       region, row, rpcTimeoutNs, operationTimeoutNs);
     S stub = stubMaker.apply(channel);
@@ -716,7 +745,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
   @Override
   public <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
-      ServiceCaller<S, R> callable, byte[] row) {
+    ServiceCaller<S, R> callable, byte[] row) {
     return coprocessorService(stubMaker, callable, null, row);
   }
 
@@ -738,9 +767,9 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
   }
 
   private <S, R> void onLocateComplete(Function<RpcChannel, S> stubMaker,
-      ServiceCaller<S, R> callable, CoprocessorCallback<R> callback, List<HRegionLocation> locs,
-      byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished,
-      AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) {
+    ServiceCaller<S, R> callable, CoprocessorCallback<R> callback, List<HRegionLocation> locs,
+    byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished,
+    AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) {
     if (error != null) {
       callback.onError(error);
       return;
@@ -769,7 +798,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
   }
 
   private final class CoprocessorServiceBuilderImpl<S, R>
-      implements CoprocessorServiceBuilder<S, R> {
+    implements CoprocessorServiceBuilder<S, R> {
 
     private final Function<RpcChannel, S> stubMaker;
 
@@ -786,7 +815,7 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
     private boolean endKeyInclusive;
 
     public CoprocessorServiceBuilderImpl(Function<RpcChannel, S> stubMaker,
-        ServiceCaller<S, R> callable, CoprocessorCallback<R> callback) {
+      ServiceCaller<S, R> callable, CoprocessorCallback<R> callback) {
       this.stubMaker = Preconditions.checkNotNull(stubMaker, "stubMaker is null");
       this.callable = Preconditions.checkNotNull(callable, "callable is null");
       this.callback = Preconditions.checkNotNull(callback, "callback is null");
@@ -823,8 +852,8 @@ class RawAsyncTableImpl implements AsyncTable<AdvancedScanResultConsumer> {
 
   @Override
   public <S, R> CoprocessorServiceBuilder<S, R> coprocessorService(
-      Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
-      CoprocessorCallback<R> callback) {
+    Function<RpcChannel, S> stubMaker, ServiceCaller<S, R> callable,
+    CoprocessorCallback<R> callback) {
     return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback);
   }
 }
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 9117fef..b671095 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -393,10 +393,11 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
   }
 
   private Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
-      final Message param, Message returnType, final User ticket,
-      final Address addr, final RpcCallback<Message> callback) {
-    Span span = TraceUtil.getGlobalTracer().spanBuilder("RpcClient.callMethod." + md.getFullName())
-      .startSpan();
+    final Message param, Message returnType, final User ticket, final Address addr,
+    final RpcCallback<Message> callback) {
+    Span span = TraceUtil.createSpan("RpcClient.callMethod")
+      .setAttribute(TraceUtil.RPC_SERVICE_KEY, md.getService().getName())
+      .setAttribute(TraceUtil.RPC_METHOD_KEY, md.getName());
     try (Scope scope = span.makeCurrent()) {
       final MetricsConnection.CallStats cs = MetricsConnection.newCallStats();
       cs.setStartTime(EnvironmentEdgeManager.currentTime());
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java
new file mode 100644
index 0000000..708ae4c
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionLocatorTracing.java
@@ -0,0 +1,157 @@
+/**
+ * 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.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertEquals;
+
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule;
+import io.opentelemetry.sdk.trace.data.SpanData;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
+@Category({ ClientTests.class, MediumTests.class })
+public class TestAsyncRegionLocatorTracing {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAsyncRegionLocatorTracing.class);
+
+  private static Configuration CONF = HBaseConfiguration.create();
+
+  private AsyncConnectionImpl conn;
+
+  private RegionLocations locs;
+
+  @Rule
+  public OpenTelemetryRule traceRule = OpenTelemetryRule.create();
+
+  @Before
+  public void setUp() throws IOException {
+    RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
+    locs = new RegionLocations(
+      new HRegionLocation(metaRegionInfo,
+        ServerName.valueOf("127.0.0.1", 12345, System.currentTimeMillis())),
+      new HRegionLocation(RegionReplicaUtil.getRegionInfoForReplica(metaRegionInfo, 1),
+        ServerName.valueOf("127.0.0.2", 12345, System.currentTimeMillis())),
+      new HRegionLocation(RegionReplicaUtil.getRegionInfoForReplica(metaRegionInfo, 2),
+        ServerName.valueOf("127.0.0.3", 12345, System.currentTimeMillis())));
+    conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF) {
+
+      @Override
+      public CompletableFuture<RegionLocations> getMetaRegionLocations() {
+        return CompletableFuture.completedFuture(locs);
+      }
+    }, "test", UserProvider.instantiate(CONF).getCurrent());
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    Closeables.close(conn, true);
+  }
+
+  private SpanData waitSpan(String name) {
+    Waiter.waitFor(CONF, 1000,
+      () -> traceRule.getSpans().stream().map(SpanData::getName).anyMatch(s -> s.equals(name)));
+    return traceRule.getSpans().stream().filter(s -> s.getName().equals(name)).findFirst().get();
+  }
+
+  @Test
+  public void testClearCache() {
+    conn.getLocator().clearCache();
+    SpanData span = waitSpan("AsyncRegionLocator.clearCache");
+    assertEquals(StatusCode.OK, span.getStatus().getStatusCode());
+  }
+
+  @Test
+  public void testClearCacheServerName() {
+    ServerName sn = ServerName.valueOf("127.0.0.1", 12345, System.currentTimeMillis());
+    conn.getLocator().clearCache(sn);
+    SpanData span = waitSpan("AsyncRegionLocator.clearCache");
+    assertEquals(StatusCode.OK, span.getStatus().getStatusCode());
+    assertEquals(sn.toString(), span.getAttributes().get(TraceUtil.SERVER_NAME_KEY));
+  }
+
+  @Test
+  public void testClearCacheTableName() {
+    conn.getLocator().clearCache(TableName.META_TABLE_NAME);
+    SpanData span = waitSpan("AsyncRegionLocator.clearCache");
+    assertEquals(StatusCode.OK, span.getStatus().getStatusCode());
+    assertEquals(TableName.META_TABLE_NAME.getNamespaceAsString(),
+      span.getAttributes().get(TraceUtil.NAMESPACE_KEY));
+    assertEquals(TableName.META_TABLE_NAME.getNameAsString(),
+      span.getAttributes().get(TraceUtil.TABLE_KEY));
+  }
+
+  @Test
+  public void testGetRegionLocation() {
+    conn.getLocator().getRegionLocation(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW,
+      RegionLocateType.CURRENT, TimeUnit.SECONDS.toNanos(1)).join();
+    SpanData span = waitSpan("AsyncRegionLocator.getRegionLocation");
+    assertEquals(StatusCode.OK, span.getStatus().getStatusCode());
+    assertEquals(TableName.META_TABLE_NAME.getNamespaceAsString(),
+      span.getAttributes().get(TraceUtil.NAMESPACE_KEY));
+    assertEquals(TableName.META_TABLE_NAME.getNameAsString(),
+      span.getAttributes().get(TraceUtil.TABLE_KEY));
+    List<String> regionNames = span.getAttributes().get(TraceUtil.REGION_NAMES_KEY);
+    assertEquals(1, regionNames.size());
+    assertEquals(locs.getDefaultRegionLocation().getRegion().getRegionNameAsString(),
+      regionNames.get(0));
+  }
+
+  @Test
+  public void testGetRegionLocations() {
+    conn.getLocator().getRegionLocations(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW,
+      RegionLocateType.CURRENT, false, TimeUnit.SECONDS.toNanos(1)).join();
+    SpanData span = waitSpan("AsyncRegionLocator.getRegionLocations");
+    assertEquals(StatusCode.OK, span.getStatus().getStatusCode());
+    assertEquals(TableName.META_TABLE_NAME.getNamespaceAsString(),
+      span.getAttributes().get(TraceUtil.NAMESPACE_KEY));
+    assertEquals(TableName.META_TABLE_NAME.getNameAsString(),
+      span.getAttributes().get(TraceUtil.TABLE_KEY));
+    List<String> regionNames = span.getAttributes().get(TraceUtil.REGION_NAMES_KEY);
+    assertEquals(3, regionNames.size());
+    for (int i = 0; i < 3; i++) {
+      assertEquals(locs.getRegionLocation(i).getRegion().getRegionNameAsString(),
+        regionNames.get(i));
+    }
+  }
+}
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java
new file mode 100644
index 0000000..07cdf0e
--- /dev/null
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java
@@ -0,0 +1,417 @@
+/*
+ * 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.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+
+import io.opentelemetry.api.trace.Span.Kind;
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule;
+import io.opentelemetry.sdk.trace.data.SpanData;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.ipc.HBaseRpcController;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
+
+@Category({ ClientTests.class, MediumTests.class })
+public class TestAsyncTableTracing {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAsyncTableTracing.class);
+
+  private static Configuration CONF = HBaseConfiguration.create();
+
+  private ClientService.Interface stub;
+
+  private AsyncConnection conn;
+
+  private AsyncTable<?> table;
+
+  @Rule
+  public OpenTelemetryRule traceRule = OpenTelemetryRule.create();
+
+  @Before
+  public void setUp() throws IOException {
+    stub = mock(ClientService.Interface.class);
+    AtomicInteger scanNextCalled = new AtomicInteger(0);
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        ScanRequest req = invocation.getArgument(1);
+        RpcCallback<ScanResponse> done = invocation.getArgument(2);
+        if (!req.hasScannerId()) {
+          done.run(ScanResponse.newBuilder().setScannerId(1).setTtl(800)
+            .setMoreResultsInRegion(true).setMoreResults(true).build());
+        } else {
+          if (req.hasCloseScanner() && req.getCloseScanner()) {
+            done.run(ScanResponse.getDefaultInstance());
+          } else {
+            Cell cell = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+              .setType(Cell.Type.Put).setRow(Bytes.toBytes(scanNextCalled.incrementAndGet()))
+              .setFamily(Bytes.toBytes("cf")).setQualifier(Bytes.toBytes("cq"))
+              .setValue(Bytes.toBytes("v")).build();
+            Result result = Result.create(Arrays.asList(cell));
+            ScanResponse.Builder builder = ScanResponse.newBuilder().setScannerId(1).setTtl(800)
+              .addResults(ProtobufUtil.toResult(result));
+            if (req.getLimitOfRows() == 1) {
+              builder.setMoreResultsInRegion(false).setMoreResults(false);
+            } else {
+              builder.setMoreResultsInRegion(true).setMoreResults(true);
+            }
+            ForkJoinPool.commonPool().execute(() -> done.run(builder.build()));
+          }
+        }
+        return null;
+      }
+    }).when(stub).scan(any(HBaseRpcController.class), any(ScanRequest.class), any());
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        ClientProtos.MultiResponse resp =
+          ClientProtos.MultiResponse.newBuilder()
+            .addRegionActionResult(RegionActionResult.newBuilder().addResultOrException(
+              ResultOrException.newBuilder().setResult(ProtobufUtil.toResult(new Result()))))
+            .build();
+        RpcCallback<ClientProtos.MultiResponse> done = invocation.getArgument(2);
+        ForkJoinPool.commonPool().execute(() -> done.run(resp));
+        return null;
+      }
+    }).when(stub).multi(any(HBaseRpcController.class), any(ClientProtos.MultiRequest.class), any());
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        MutationProto req = ((MutateRequest) invocation.getArgument(1)).getMutation();
+        MutateResponse resp;
+        switch (req.getMutateType()) {
+          case INCREMENT:
+            ColumnValue value = req.getColumnValue(0);
+            QualifierValue qvalue = value.getQualifierValue(0);
+            Cell cell = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
+              .setType(Cell.Type.Put).setRow(req.getRow().toByteArray())
+              .setFamily(value.getFamily().toByteArray())
+              .setQualifier(qvalue.getQualifier().toByteArray())
+              .setValue(qvalue.getValue().toByteArray()).build();
+            resp = MutateResponse.newBuilder()
+              .setResult(ProtobufUtil.toResult(Result.create(Arrays.asList(cell)))).build();
+            break;
+          default:
+            resp = MutateResponse.getDefaultInstance();
+            break;
+        }
+        RpcCallback<MutateResponse> done = invocation.getArgument(2);
+        ForkJoinPool.commonPool().execute(() -> done.run(resp));
+        return null;
+      }
+    }).when(stub).mutate(any(HBaseRpcController.class), any(MutateRequest.class), any());
+    doAnswer(new Answer<Void>() {
+
+      @Override
+      public Void answer(InvocationOnMock invocation) throws Throwable {
+        RpcCallback<GetResponse> done = invocation.getArgument(2);
+        ForkJoinPool.commonPool().execute(() -> done.run(GetResponse.getDefaultInstance()));
+        return null;
+      }
+    }).when(stub).get(any(HBaseRpcController.class), any(GetRequest.class), any());
+    conn = new AsyncConnectionImpl(CONF, new DoNothingConnectionRegistry(CONF), "test",
+      UserProvider.instantiate(CONF).getCurrent()) {
+
+      @Override
+      AsyncRegionLocator getLocator() {
+        AsyncRegionLocator locator = mock(AsyncRegionLocator.class);
+        Answer<CompletableFuture<HRegionLocation>> answer =
+          new Answer<CompletableFuture<HRegionLocation>>() {
+
+            @Override
+            public CompletableFuture<HRegionLocation> answer(InvocationOnMock invocation)
+              throws Throwable {
+              TableName tableName = invocation.getArgument(0);
+              RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
+              ServerName serverName = ServerName.valueOf("rs", 16010, 12345);
+              HRegionLocation loc = new HRegionLocation(info, serverName);
+              return CompletableFuture.completedFuture(loc);
+            }
+          };
+        doAnswer(answer).when(locator).getRegionLocation(any(TableName.class), any(byte[].class),
+          any(RegionLocateType.class), anyLong());
+        doAnswer(answer).when(locator).getRegionLocation(any(TableName.class), any(byte[].class),
+          anyInt(), any(RegionLocateType.class), anyLong());
+        return locator;
+      }
+
+      @Override
+      ClientService.Interface getRegionServerStub(ServerName serverName) throws IOException {
+        return stub;
+      }
+    };
+    table = conn.getTable(TableName.valueOf("table"), ForkJoinPool.commonPool());
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    Closeables.close(conn, true);
+  }
+
+  private void assertTrace(String methodName) {
+    Waiter.waitFor(CONF, 1000,
+      () -> traceRule.getSpans().stream()
+        .anyMatch(span -> span.getName().equals("AsyncTable." + methodName) &&
+          span.getKind() == Kind.INTERNAL && span.hasEnded()));
+    SpanData data = traceRule.getSpans().stream()
+      .filter(s -> s.getName().equals("AsyncTable." + methodName)).findFirst().get();
+    assertEquals(StatusCode.OK, data.getStatus().getStatusCode());
+    TableName tableName = table.getName();
+    assertEquals(tableName.getNamespaceAsString(),
+      data.getAttributes().get(TraceUtil.NAMESPACE_KEY));
+    assertEquals(tableName.getNameAsString(), data.getAttributes().get(TraceUtil.TABLE_KEY));
+  }
+
+  @Test
+  public void testExists() {
+    table.exists(new Get(Bytes.toBytes(0))).join();
+    assertTrace("get");
+  }
+
+  @Test
+  public void testGet() {
+    table.get(new Get(Bytes.toBytes(0))).join();
+    assertTrace("get");
+  }
+
+  @Test
+  public void testPut() {
+    table.put(new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"),
+      Bytes.toBytes("v"))).join();
+    assertTrace("put");
+  }
+
+  @Test
+  public void testDelete() {
+    table.delete(new Delete(Bytes.toBytes(0))).join();
+    assertTrace("delete");
+  }
+
+  @Test
+  public void testAppend() {
+    table.append(new Append(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"),
+      Bytes.toBytes("v"))).join();
+    assertTrace("append");
+  }
+
+  @Test
+  public void testIncrement() {
+    table
+      .increment(
+        new Increment(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1))
+      .join();
+    assertTrace("increment");
+  }
+
+  @Test
+  public void testIncrementColumnValue1() {
+    table.incrementColumnValue(Bytes.toBytes(0), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1)
+      .join();
+    assertTrace("increment");
+  }
+
+  @Test
+  public void testIncrementColumnValue2() {
+    table.incrementColumnValue(Bytes.toBytes(0), Bytes.toBytes("cf"), Bytes.toBytes("cq"), 1,
+      Durability.ASYNC_WAL).join();
+    assertTrace("increment");
+  }
+
+  @Test
+  public void testCheckAndMutate() {
+    table.checkAndMutate(CheckAndMutate.newBuilder(Bytes.toBytes(0))
+      .ifEquals(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))
+      .build(new Delete(Bytes.toBytes(0)))).join();
+    assertTrace("checkAndMutate");
+  }
+
+  @Test
+  public void testCheckAndMutateList() {
+    CompletableFuture
+      .allOf(table.checkAndMutate(Arrays.asList(CheckAndMutate.newBuilder(Bytes.toBytes(0))
+        .ifEquals(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))
+        .build(new Delete(Bytes.toBytes(0))))).toArray(new CompletableFuture[0]))
+      .join();
+    assertTrace("checkAndMutateList");
+  }
+
+  @Test
+  public void testCheckAndMutateAll() {
+    table.checkAndMutateAll(Arrays.asList(CheckAndMutate.newBuilder(Bytes.toBytes(0))
+      .ifEquals(Bytes.toBytes("cf"), Bytes.toBytes("cq"), Bytes.toBytes("v"))
+      .build(new Delete(Bytes.toBytes(0))))).join();
+    assertTrace("checkAndMutateList");
+  }
+
+  @Test
+  public void testMutateRow() throws Exception {
+    byte[] row = Bytes.toBytes(0);
+    RowMutations mutation = new RowMutations(row);
+    mutation.add(new Delete(row));
+    table.mutateRow(mutation).get();
+    assertTrace("mutateRow");
+  }
+
+  @Test
+  public void testScanAll() throws IOException {
+    table.scanAll(new Scan().setCaching(1).setMaxResultSize(1).setLimit(1)).join();
+    assertTrace("scanAll");
+  }
+
+  @Test
+  public void testExistsList() {
+    CompletableFuture
+      .allOf(
+        table.exists(Arrays.asList(new Get(Bytes.toBytes(0)))).toArray(new CompletableFuture[0]))
+      .join();
+    assertTrace("getList");
+  }
+
+  @Test
+  public void testExistsAll() {
+    table.existsAll(Arrays.asList(new Get(Bytes.toBytes(0)))).join();
+    assertTrace("getList");
+  }
+
+  @Test
+  public void testGetList() {
+    CompletableFuture
+      .allOf(table.get(Arrays.asList(new Get(Bytes.toBytes(0)))).toArray(new CompletableFuture[0]))
+      .join();
+    assertTrace("getList");
+  }
+
+  @Test
+  public void testGetAll() {
+    table.getAll(Arrays.asList(new Get(Bytes.toBytes(0)))).join();
+    assertTrace("getList");
+  }
+
+  @Test
+  public void testPutList() {
+    CompletableFuture
+      .allOf(table.put(Arrays.asList(new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"),
+        Bytes.toBytes("cq"), Bytes.toBytes("v")))).toArray(new CompletableFuture[0]))
+      .join();
+    assertTrace("putList");
+  }
+
+  @Test
+  public void testPutAll() {
+    table.putAll(Arrays.asList(new Put(Bytes.toBytes(0)).addColumn(Bytes.toBytes("cf"),
+      Bytes.toBytes("cq"), Bytes.toBytes("v")))).join();
+    assertTrace("putList");
+  }
+
+  @Test
+  public void testDeleteList() {
+    CompletableFuture
+      .allOf(
+        table.delete(Arrays.asList(new Delete(Bytes.toBytes(0)))).toArray(new CompletableFuture[0]))
+      .join();
+    assertTrace("deleteList");
+  }
+
+  @Test
+  public void testDeleteAll() {
+    table.deleteAll(Arrays.asList(new Delete(Bytes.toBytes(0)))).join();
+    assertTrace("deleteList");
+  }
+
+  @Test
+  public void testBatch() {
+    CompletableFuture
+      .allOf(
+        table.batch(Arrays.asList(new Delete(Bytes.toBytes(0)))).toArray(new CompletableFuture[0]))
+      .join();
+    assertTrace("batch");
+  }
+
+  @Test
+  public void testBatchAll() {
+    table.batchAll(Arrays.asList(new Delete(Bytes.toBytes(0)))).join();
+    assertTrace("batch");
+  }
+
+  @Test
+  public void testConnClose() throws IOException {
+    conn.close();
+    Waiter.waitFor(CONF, 1000,
+      () -> traceRule.getSpans().stream()
+        .anyMatch(span -> span.getName().equals("AsyncConnection.close") &&
+          span.getKind() == Kind.INTERNAL && span.hasEnded()));
+    SpanData data = traceRule.getSpans().stream()
+      .filter(s -> s.getName().equals("AsyncConnection.close")).findFirst().get();
+    assertEquals(StatusCode.OK, data.getStatus().getStatusCode());
+  }
+}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
index 768de9c..d0da071 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
@@ -18,7 +18,19 @@
 package org.apache.hadoop.hbase.trace;
 
 import io.opentelemetry.api.GlobalOpenTelemetry;
+import io.opentelemetry.api.common.AttributeKey;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.Span.Kind;
+import io.opentelemetry.api.trace.StatusCode;
 import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.api.trace.attributes.SemanticAttributes;
+import io.opentelemetry.context.Context;
+import io.opentelemetry.context.Scope;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.FutureUtils;
 import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
@@ -26,10 +38,132 @@ public final class TraceUtil {
 
   private static final String INSTRUMENTATION_NAME = "io.opentelemetry.contrib.hbase";
 
+  public static final AttributeKey<String> NAMESPACE_KEY = SemanticAttributes.DB_HBASE_NAMESPACE;
+
+  public static final AttributeKey<String> TABLE_KEY = AttributeKey.stringKey("db.hbase.table");
+
+  public static final AttributeKey<List<String>> REGION_NAMES_KEY =
+    AttributeKey.stringArrayKey("db.hbase.regions");
+
+  public static final AttributeKey<String> RPC_SERVICE_KEY =
+    AttributeKey.stringKey("db.hbase.rpc.service");
+
+  public static final AttributeKey<String> RPC_METHOD_KEY =
+    AttributeKey.stringKey("db.hbase.rpc.method");
+
+  public static final AttributeKey<String> SERVER_NAME_KEY =
+    AttributeKey.stringKey("db.hbase.server.name");
+
   private TraceUtil() {
   }
 
   public static Tracer getGlobalTracer() {
     return GlobalOpenTelemetry.getTracer(INSTRUMENTATION_NAME);
   }
+
+  /**
+   * Create a {@link Kind#INTERNAL} span.
+   */
+  public static Span createSpan(String name) {
+    return createSpan(name, Kind.INTERNAL);
+  }
+
+  /**
+   * Create a {@link Kind#INTERNAL} span and set table related attributes.
+   */
+  public static Span createTableSpan(String spanName, TableName tableName) {
+    return createSpan(spanName).setAttribute(NAMESPACE_KEY, tableName.getNamespaceAsString())
+      .setAttribute(TABLE_KEY, tableName.getNameAsString());
+  }
+
+  /**
+   * Create a span with the given {@code kind}. Notice that, OpenTelemetry only expects one
+   * {@link Kind#CLIENT} span and one {@link Kind#SERVER} span for a traced request, so use this
+   * with caution when you want to create spans with kind other than {@link Kind#INTERNAL}.
+   */
+  private static Span createSpan(String name, Kind kind) {
+    return getGlobalTracer().spanBuilder(name).setSpanKind(kind).startSpan();
+  }
+
+  /**
+   * Create a span which parent is from remote, i.e, passed through rpc.
+   * </p>
+   * We will set the kind of the returned span to {@link Kind#SERVER}, as this should be the top
+   * most span at server side.
+   */
+  public static Span createRemoteSpan(String name, Context ctx) {
+    return getGlobalTracer().spanBuilder(name).setParent(ctx).setSpanKind(Kind.SERVER).startSpan();
+  }
+
+  /**
+   * Trace an asynchronous operation for a table.
+   */
+  public static <T> CompletableFuture<T> tracedFuture(Supplier<CompletableFuture<T>> action,
+    String spanName, TableName tableName) {
+    Span span = createTableSpan(spanName, tableName);
+    try (Scope scope = span.makeCurrent()) {
+      CompletableFuture<T> future = action.get();
+      endSpan(future, span);
+      return future;
+    }
+  }
+
+  /**
+   * Trace an asynchronous operation.
+   */
+  public static <T> CompletableFuture<T> tracedFuture(Supplier<CompletableFuture<T>> action,
+    String spanName) {
+    Span span = createSpan(spanName);
+    try (Scope scope = span.makeCurrent()) {
+      CompletableFuture<T> future = action.get();
+      endSpan(future, span);
+      return future;
+    }
+  }
+
+  /**
+   * Trace an asynchronous operation, and finish the create {@link Span} when all the given
+   * {@code futures} are completed.
+   */
+  public static <T> List<CompletableFuture<T>> tracedFutures(
+    Supplier<List<CompletableFuture<T>>> action, String spanName, TableName tableName) {
+    Span span = createTableSpan(spanName, tableName);
+    try (Scope scope = span.makeCurrent()) {
+      List<CompletableFuture<T>> futures = action.get();
+      endSpan(CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])), span);
+      return futures;
+    }
+  }
+
+  /**
+   * Finish the {@code span} when the given {@code future} is completed.
+   */
+  private static void endSpan(CompletableFuture<?> future, Span span) {
+    FutureUtils.addListener(future, (resp, error) -> {
+      if (error != null) {
+        span.recordException(error);
+        span.setStatus(StatusCode.ERROR);
+      } else {
+        span.setStatus(StatusCode.OK);
+      }
+      span.end();
+    });
+  }
+
+  public static void trace(Runnable action, String spanName) {
+    trace(action, () -> createSpan(spanName));
+  }
+
+  public static void trace(Runnable action, Supplier<Span> creator) {
+    Span span = creator.get();
+    try (Scope scope = span.makeCurrent()) {
+      action.run();
+      span.setStatus(StatusCode.OK);
+    } catch (Throwable e) {
+      span.recordException(e);
+      span.setStatus(StatusCode.ERROR);
+    } finally {
+      span.end();
+    }
+  }
 }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 203f079..7cc1d2b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -122,9 +122,10 @@ public class CallRunner {
       RpcServer.CurCall.set(call);
       String serviceName = getServiceName();
       String methodName = getMethodName();
-      String traceString = serviceName + "." + methodName;
-      Span span = TraceUtil.getGlobalTracer().spanBuilder(traceString)
-        .setParent(Context.current().with(((ServerCall<?>) call).getSpan())).startSpan();
+      Span span = TraceUtil.getGlobalTracer().spanBuilder("RpcServer.callMethod")
+        .setParent(Context.current().with(((ServerCall<?>) call).getSpan())).startSpan()
+        .setAttribute(TraceUtil.RPC_SERVICE_KEY, serviceName)
+        .setAttribute(TraceUtil.RPC_METHOD_KEY, methodName);
       try (Scope traceScope = span.makeCurrent()) {
         if (!this.rpcServer.isStarted()) {
           InetSocketAddress address = rpcServer.getListenerAddress();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
index db7f052..823005b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
@@ -629,8 +629,7 @@ abstract class ServerRpcConnection implements Closeable {
     };
     Context traceCtx = GlobalOpenTelemetry.getPropagators().getTextMapPropagator()
       .extract(Context.current(), header.getTraceInfo(), getter);
-    Span span =
-      TraceUtil.getGlobalTracer().spanBuilder("RpcServer.process").setParent(traceCtx).startSpan();
+    Span span = TraceUtil.createRemoteSpan("RpcServer.process", traceCtx);
     try (Scope scope = span.makeCurrent()) {
       int id = header.getCallId();
       if (RpcServer.LOG.isTraceEnabled()) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index 11978ca..4aca764 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Server;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
+import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.util.StringUtils;
@@ -448,6 +449,19 @@ public abstract class AbstractTestIPC {
     }
   }
 
+  private SpanData waitSpan(String name) {
+    Waiter.waitFor(CONF, 1000,
+      () -> traceRule.getSpans().stream().map(SpanData::getName).anyMatch(s -> s.equals(name)));
+    return traceRule.getSpans().stream().filter(s -> s.getName().equals(name)).findFirst().get();
+  }
+
+  private void assertRpcAttribute(SpanData data, String methodName) {
+    assertEquals(SERVICE.getDescriptorForType().getName(),
+      data.getAttributes().get(TraceUtil.RPC_SERVICE_KEY));
+    assertEquals(methodName,
+      data.getAttributes().get(TraceUtil.RPC_METHOD_KEY));
+  }
+
   @Test
   public void testTracing() throws IOException, ServiceException {
     RpcServer rpcServer = createRpcServer(null, "testRpcServer",
@@ -457,9 +471,8 @@ public abstract class AbstractTestIPC {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       stub.pause(null, PauseRequestProto.newBuilder().setMs(100).build());
-      Waiter.waitFor(CONF, 1000, () -> traceRule.getSpans().stream().map(SpanData::getName)
-        .anyMatch(s -> s.equals("RpcClient.callMethod.TestProtobufRpcProto.pause")));
-
+      assertRpcAttribute(waitSpan("RpcClient.callMethod"), "pause");
+      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "pause");
       assertSameTraceId();
       for (SpanData data : traceRule.getSpans()) {
         assertThat(
@@ -471,9 +484,8 @@ public abstract class AbstractTestIPC {
       traceRule.clearSpans();
       assertThrows(ServiceException.class,
         () -> stub.error(null, EmptyRequestProto.getDefaultInstance()));
-      Waiter.waitFor(CONF, 1000, () -> traceRule.getSpans().stream().map(SpanData::getName)
-        .anyMatch(s -> s.equals("RpcClient.callMethod.TestProtobufRpcProto.error")));
-
+      assertRpcAttribute(waitSpan("RpcClient.callMethod"), "error");
+      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "error");
       assertSameTraceId();
       for (SpanData data : traceRule.getSpans()) {
         assertEquals(StatusCode.ERROR, data.getStatus().getStatusCode());
diff --git a/pom.xml b/pom.xml
index 95bb712..9fcf86b 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1484,7 +1484,6 @@
     <junit.version>4.13</junit.version>
     <hamcrest.version>1.3</hamcrest.version>
     <opentelemetry.version>0.13.1</opentelemetry.version>
-    <opentelemetry-instrumentation.version>0.13.0</opentelemetry-instrumentation.version>
     <log4j.version>1.2.17</log4j.version>
     <mockito-core.version>2.28.2</mockito-core.version>
     <!--Internally we use a different version of protobuf. See hbase-protocol-shaded-->
@@ -2193,7 +2192,7 @@
       <dependency>
         <groupId>io.opentelemetry.javaagent</groupId>
         <artifactId>opentelemetry-javaagent</artifactId>
-        <version>${opentelemetry-instrumentation.version}</version>
+        <version>${opentelemetry.version}</version>
         <classifier>all</classifier>
       </dependency>
       <dependency>

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

Posted by ta...@apache.org.
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 <ta...@apache.org>
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 <zh...@apache.org>
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 .../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<Hbck> 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<Hbck>() {
+
+      @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 0000000..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.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import static org.junit.Assert.assertEquals;
+
+import io.opentelemetry.api.trace.SpanKind;
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule;
+import io.opentelemetry.sdk.trace.data.SpanData;
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Waiter;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.testclassification.ClientTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
+@Category({ ClientTests.class, MediumTests.class })
+public class TestAsyncConnectionTracing {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestAsyncConnectionTracing.class);
+
+  private static Configuration CONF = HBaseConfiguration.create();
+
+  private ServerName masterServer =
+    ServerName.valueOf("localhost", 12345, System.currentTimeMillis());
+
+  private AsyncConnection conn;
+
+  @Rule
+  public OpenTelemetryRule traceRule = OpenTelemetryRule.create();
+
+  @Before
+  public void setUp() throws IOException {
+    ConnectionRegistry registry = new DoNothingConnectionRegistry(CONF) {
+
+      @Override
+      public CompletableFuture<ServerName> getActiveMaster() {
+        return CompletableFuture.completedFuture(masterServer);
+      }
+    };
+    conn = new AsyncConnectionImpl(CONF, registry, "test",
+      UserProvider.instantiate(CONF).getCurrent());
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    Closeables.close(conn, true);
+  }
+
+  private void assertTrace(String methodName, ServerName serverName) {
+    Waiter.waitFor(CONF, 1000,
+      () -> traceRule.getSpans().stream()
+        .anyMatch(span -> span.getName().equals("AsyncConnection." + methodName) &&
+          span.getKind() == SpanKind.INTERNAL && span.hasEnded()));
+    SpanData data = traceRule.getSpans().stream()
+      .filter(s -> s.getName().equals("AsyncConnection." + methodName)).findFirst().get();
+    assertEquals(StatusCode.OK, data.getStatus().getStatusCode());
+    if (serverName != null) {
+      assertEquals(serverName.getServerName(), data.getAttributes().get(TraceUtil.SERVER_NAME_KEY));
+    }
+  }
+
+  @Test
+  public void testHbck() {
+    conn.getHbck().join();
+    assertTrace("getHbck", masterServer);
+  }
+
+  @Test
+  public void testHbckWithServerName() throws IOException {
+    ServerName serverName = ServerName.valueOf("localhost", 23456, System.currentTimeMillis());
+    conn.getHbck(serverName);
+    assertTrace("getHbck", serverName);
+  }
+
+  @Test
+  public void testClose() throws IOException {
+    conn.close();
+    assertTrace("close", null);
+  }
+}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
index 43c2e04..fb37080 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
@@ -189,6 +189,20 @@ public final class TraceUtil {
     }
   }
 
+  public static <T> T trace(Supplier<T> action, String spanName) {
+    Span span = createSpan(spanName);
+    try (Scope scope = span.makeCurrent()) {
+      T ret = action.get();
+      span.setStatus(StatusCode.OK);
+      return ret;
+    } catch (Throwable e) {
+      setError(span, e);
+      throw e;
+    } finally {
+      span.end();
+    }
+  }
+
   @FunctionalInterface
   public interface IOExceptionCallable<V> {
     V call() throws IOException;

[hbase] 18/18: HBASE-26168 Backport HBASE-25811 "The client integration test is failing after HBASE-22120 merged" into branch-2 (#3634)

Posted by ta...@apache.org.
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 625093f907b97a7f51946862843edbd6a7c57e8d
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Fri Aug 27 09:09:55 2021 -0700

    HBASE-26168 Backport HBASE-25811 "The client integration test is failing after HBASE-22120 merged" into branch-2 (#3634)
    
    move opentelemetry jars to client-facing-thirdparty
    add opentelemetry jars when init map reduce job dependencies
    
    original commit 8d2a0efb7a9c17e58892369b281b4af49a5f0d7e
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 hbase-assembly/src/main/assembly/client.xml                       | 8 ++++++++
 hbase-assembly/src/main/assembly/hadoop-three-compat.xml          | 1 +
 .../org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java     | 4 +++-
 3 files changed, 12 insertions(+), 1 deletion(-)

diff --git a/hbase-assembly/src/main/assembly/client.xml b/hbase-assembly/src/main/assembly/client.xml
index bd65cb4..48940b7 100644
--- a/hbase-assembly/src/main/assembly/client.xml
+++ b/hbase-assembly/src/main/assembly/client.xml
@@ -65,6 +65,7 @@
               <exclude>org.slf4j:jcl-over-slf4j</exclude>
               <exclude>org.slf4j:jul-to-slf4j</exclude>
               <exclude>org.slf4j:slf4j-log4j12</exclude>
+              <exclude>io.opentelemetry.javaagent:*</exclude>
             </excludes>
           </dependencySet>
         </dependencySets>
@@ -153,6 +154,13 @@
         <include>org.slf4j:jcl-over-slf4j</include>
         <include>org.slf4j:jul-to-slf4j</include>
         <include>org.slf4j:slf4j-log4j12</include>
+        <include>io.opentelemetry:*</include>
+      </includes>
+    </dependencySet>
+    <dependencySet>
+      <outputDirectory>lib/trace</outputDirectory>
+      <includes>
+        <include>io.opentelemetry.javaagent:*</include>
       </includes>
     </dependencySet>
   </dependencySets>
diff --git a/hbase-assembly/src/main/assembly/hadoop-three-compat.xml b/hbase-assembly/src/main/assembly/hadoop-three-compat.xml
index 9780444..6c670a7 100644
--- a/hbase-assembly/src/main/assembly/hadoop-three-compat.xml
+++ b/hbase-assembly/src/main/assembly/hadoop-three-compat.xml
@@ -213,6 +213,7 @@
         <include>org.slf4j:jcl-over-slf4j</include>
         <include>org.slf4j:jul-to-slf4j</include>
         <include>org.slf4j:slf4j-log4j12</include>
+        <include>io.opentelemetry:*</include>
       </includes>
     </dependencySet>
     <dependencySet>
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
index d89c5b5..eaf1b40 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java
@@ -832,7 +832,9 @@ public class TableMapReduceUtil {
       org.apache.zookeeper.ZooKeeper.class,                          // zookeeper
       com.google.protobuf.Message.class,                             // protobuf
       com.codahale.metrics.MetricRegistry.class,                     // metrics-core
-      org.apache.commons.lang3.ArrayUtils.class);                    // commons-lang
+      org.apache.commons.lang3.ArrayUtils.class,                     // commons-lang
+      io.opentelemetry.api.trace.Span.class,                         // opentelemetry-api
+      io.opentelemetry.semconv.trace.attributes.SemanticAttributes.class); // opentelemetry-semconv
   }
 
   /**

[hbase] 03/18: HBASE-26126 Backport HBASE-25424 "Find a way to config OpenTelemetry tracing without directly depending on opentelemetry-sdk" to branch-2 (#3552)

Posted by ta...@apache.org.
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 af460132c09964e1ea2ff858182ef662ee973bc9
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Tue Aug 3 09:54:37 2021 -0700

    HBASE-26126 Backport HBASE-25424 "Find a way to config OpenTelemetry tracing without directly depending on opentelemetry-sdk" to branch-2 (#3552)
    
    3/17 commits of HBASE-22120, original commit#57960fa8fa7228d65b1a4adc8e9b5b1a8158824d
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 bin/hbase                                                | 10 ++++++++++
 conf/hbase-env.sh                                        |  5 +++++
 hbase-assembly/pom.xml                                   |  6 ++++++
 hbase-assembly/src/main/assembly/hadoop-three-compat.xml |  7 +++++++
 pom.xml                                                  |  7 +++++++
 5 files changed, 35 insertions(+)

diff --git a/bin/hbase b/bin/hbase
index 496ebe1..5f3b326 100755
--- a/bin/hbase
+++ b/bin/hbase
@@ -487,6 +487,11 @@ add_jdk11_deps_to_classpath() {
   done
 }
 
+enable_trace() {
+  agent_jar=$(find lib/trace -type f -name "opentelemetry-javaagent-*")
+  HBASE_OPTS="$HBASE_OPTS -javaagent:$agent_jar $HBASE_TRACE_OPTS"
+}
+
 #Add the development env class path stuff
 if $in_dev_env; then
   add_maven_deps_to_classpath "cached_classpath.txt"
@@ -764,6 +769,11 @@ elif [ "${DEBUG}" = "true" ]; then
   echo "JDK11 jars skipped from classpath."
 fi
 
+if [[ -n "${HBASE_TRACE_OPTS}" ]]; then
+  echo "Attach opentelemetry agent to enable trace"
+  enable_trace
+fi
+
 # Have JVM dump heap if we run out of memory.  Files will be 'launch directory'
 # and are named like the following: java_pid21612.hprof. Apparently it doesn't
 # 'cost' to have this flag enabled. Its a 1.6 flag only. See:
diff --git a/conf/hbase-env.sh b/conf/hbase-env.sh
index 6cc16d8..b3c4baf 100644
--- a/conf/hbase-env.sh
+++ b/conf/hbase-env.sh
@@ -144,3 +144,8 @@
 
 # Additional argments passed to jshell invocation
 # export HBASE_JSHELL_ARGS="--startup DEFAULT --startup PRINTING --startup hbase_startup.jsh"
+
+# Uncomment to enable trace, you can change the options to use other exporters such as jaeger or
+# zipkin. See https://github.com/open-telemetry/opentelemetry-java-instrumentation on how to config
+# exporters and other components through system properties.
+# export HBASE_TRACE_OPTS="-Dotel.config.sampler.probability=0.1 -Dotel.exporter=logging"
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index 451e9f3..4ea650d 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -339,6 +339,12 @@
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
     </dependency>
+    <!-- Include OpenTelemetry agent -->
+    <dependency>
+      <groupId>io.opentelemetry.javaagent</groupId>
+      <artifactId>opentelemetry-javaagent</artifactId>
+      <classifier>all</classifier>
+    </dependency>
   </dependencies>
   <profiles>
     <profile>
diff --git a/hbase-assembly/src/main/assembly/hadoop-three-compat.xml b/hbase-assembly/src/main/assembly/hadoop-three-compat.xml
index e5422f3..9780444 100644
--- a/hbase-assembly/src/main/assembly/hadoop-three-compat.xml
+++ b/hbase-assembly/src/main/assembly/hadoop-three-compat.xml
@@ -112,6 +112,7 @@
               <exclude>org.apache.yetus:audience-annotations</exclude>
               <exclude>org.slf4j:slf4j-api</exclude>
               <exclude>org.slf4j:slf4j-log4j12</exclude>
+              <exclude>io.opentelemetry.javaagent:*</exclude>
             </excludes>
           </dependencySet>
         </dependencySets>
@@ -256,6 +257,12 @@
         <include>jakarta.jws:jakarta.jws-api</include>
       </includes>
     </dependencySet>
+    <dependencySet>
+      <outputDirectory>lib/trace</outputDirectory>
+      <includes>
+        <include>io.opentelemetry.javaagent:*</include>
+      </includes>
+    </dependencySet>
   </dependencySets>
 
 </assembly>
diff --git a/pom.xml b/pom.xml
index 8cf9b02..95bb712 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1484,6 +1484,7 @@
     <junit.version>4.13</junit.version>
     <hamcrest.version>1.3</hamcrest.version>
     <opentelemetry.version>0.13.1</opentelemetry.version>
+    <opentelemetry-instrumentation.version>0.13.0</opentelemetry-instrumentation.version>
     <log4j.version>1.2.17</log4j.version>
     <mockito-core.version>2.28.2</mockito-core.version>
     <!--Internally we use a different version of protobuf. See hbase-protocol-shaded-->
@@ -2190,6 +2191,12 @@
         <version>${opentelemetry.version}</version>
       </dependency>
       <dependency>
+        <groupId>io.opentelemetry.javaagent</groupId>
+        <artifactId>opentelemetry-javaagent</artifactId>
+        <version>${opentelemetry-instrumentation.version}</version>
+        <classifier>all</classifier>
+      </dependency>
+      <dependency>
         <groupId>com.lmax</groupId>
         <artifactId>disruptor</artifactId>
         <version>${disruptor.version}</version>

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

Posted by ta...@apache.org.
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 7ecf54f365fadf87f1ab1d90b21f094ce86646c4
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

[hbase] 07/18: HBASE-26130 Backport HBASE-25455 "Add trace support for HRegion read/… (#3594)

Posted by ta...@apache.org.
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 321dcd4e7746b7660ca0c20c07fbfeed9ca4eb69
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Tue Aug 17 22:37:41 2021 -0700

    HBASE-26130 Backport HBASE-25455 "Add trace support for HRegion read/… (#3594)
    
    7/17 commits of HBASE-22120, original commit 03e12bfa4ad62ecc6eee6a2c68d431bea2d5c473
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by:  Duo Zhang <zh...@apache.org>
---
 .../hadoop/hbase/client/AsyncRegionLocator.java    |   3 +-
 .../apache/hadoop/hbase/ipc/AbstractRpcClient.java |   3 +-
 .../org/apache/hadoop/hbase/trace/TraceUtil.java   |  40 ++++-
 .../org/apache/hadoop/hbase/ipc/CallRunner.java    |   6 +-
 .../org/apache/hadoop/hbase/ipc/ServerCall.java    |   4 +-
 .../apache/hadoop/hbase/regionserver/HRegion.java  | 185 +++++++++++---------
 .../hbase/regionserver/RegionScannerImpl.java      | 101 ++++++-----
 .../hbase/regionserver/TestAtomicOperation.java    |   2 +-
 .../hbase/regionserver/TestHRegionTracing.java     | 187 +++++++++++++++++++++
 9 files changed, 393 insertions(+), 138 deletions(-)

diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index 1d0efcc..3067785 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -102,8 +102,7 @@ class AsyncRegionLocator {
       CompletableFuture<T> future = action.get();
       FutureUtils.addListener(future, (resp, error) -> {
         if (error != null) {
-          span.recordException(error);
-          span.setStatus(StatusCode.ERROR);
+          TraceUtil.setError(span, error);
         } else {
           List<String> regionNames = getRegionNames.apply(resp);
           if (!regionNames.isEmpty()) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
index 878f9cf..fa7dfb1 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AbstractRpcClient.java
@@ -424,8 +424,7 @@ public abstract class AbstractRpcClient<T extends RpcConnection> implements RpcC
               onCallFinished(call, hrc, addr, callback);
             } finally {
               if (hrc.failed()) {
-                span.setStatus(StatusCode.ERROR);
-                span.recordException(hrc.getFailed());
+                TraceUtil.setError(span, hrc.getFailed());
               } else {
                 span.setStatus(StatusCode.OK);
               }
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
index 8eb2399..ea16df1 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
@@ -26,6 +26,7 @@ import io.opentelemetry.api.trace.Tracer;
 import io.opentelemetry.api.trace.attributes.SemanticAttributes;
 import io.opentelemetry.context.Context;
 import io.opentelemetry.context.Scope;
+import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Supplier;
@@ -58,6 +59,9 @@ public final class TraceUtil {
 
   public static final AttributeKey<Long> REMOTE_PORT_KEY = SemanticAttributes.NET_PEER_PORT;
 
+  public static final AttributeKey<Boolean> ROW_LOCK_READ_LOCK_KEY =
+    AttributeKey.booleanKey("db.hbase.rowlock.readlock");
+
   private TraceUtil() {
   }
 
@@ -139,14 +143,18 @@ public final class TraceUtil {
     }
   }
 
+  public static void setError(Span span, Throwable error) {
+    span.recordException(error);
+    span.setStatus(StatusCode.ERROR);
+  }
+
   /**
    * Finish the {@code span} when the given {@code future} is completed.
    */
   private static void endSpan(CompletableFuture<?> future, Span span) {
     FutureUtils.addListener(future, (resp, error) -> {
       if (error != null) {
-        span.recordException(error);
-        span.setStatus(StatusCode.ERROR);
+        setError(span, error);
       } else {
         span.setStatus(StatusCode.OK);
       }
@@ -164,8 +172,32 @@ public final class TraceUtil {
       action.run();
       span.setStatus(StatusCode.OK);
     } catch (Throwable e) {
-      span.recordException(e);
-      span.setStatus(StatusCode.ERROR);
+      setError(span, e);
+      throw e;
+    } finally {
+      span.end();
+    }
+  }
+
+  @FunctionalInterface
+  public interface IOExceptionCallable<V> {
+    V call() throws IOException;
+  }
+
+  public static <T> T trace(IOExceptionCallable<T> callable, String spanName) throws IOException {
+    return trace(callable, () -> createSpan(spanName));
+  }
+
+  public static <T> T trace(IOExceptionCallable<T> callable, Supplier<Span> creator)
+    throws IOException {
+    Span span = creator.get();
+    try (Scope scope = span.makeCurrent()) {
+      T ret = callable.call();
+      span.setStatus(StatusCode.OK);
+      return ret;
+    } catch (Throwable e) {
+      setError(span, e);
+      throw e;
     } finally {
       span.end();
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 7cc1d2b..6e2c37e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -136,12 +136,10 @@ public class CallRunner {
         resultPair = this.rpcServer.call(call, this.status);
       } catch (TimeoutIOException e){
         RpcServer.LOG.warn("Can not complete this request in time, drop it: " + call);
-        span.recordException(e);
-        span.setStatus(StatusCode.ERROR);
+        TraceUtil.setError(span, e);
         return;
       } catch (Throwable e) {
-        span.recordException(e);
-        span.setStatus(StatusCode.ERROR);
+        TraceUtil.setError(span, e);
         if (e instanceof ServerNotRunningYetException) {
           // If ServerNotRunningYetException, don't spew stack trace.
           if (RpcServer.LOG.isTraceEnabled()) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java
index 4a021ce..53101c9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.io.ByteBuffAllocator;
 import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.ipc.RpcServer.CallCleanup;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -232,8 +233,7 @@ public abstract class ServerCall<T extends ServerRpcConnection> implements RpcCa
     }
     if (t != null) {
       this.isError = true;
-      span.recordException(t);
-      span.setStatus(StatusCode.ERROR);
+      TraceUtil.setError(span, t);
     } else {
       span.setStatus(StatusCode.OK);
     }
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index dac4e02..6ab6f14 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -22,7 +22,6 @@ import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_K
 import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
 import edu.umd.cs.findbugs.annotations.Nullable;
 import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -3111,24 +3110,26 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   private RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners,
-      long nonceGroup, long nonce) throws IOException {
-    startRegionOperation(Operation.SCAN);
-    try {
-      // Verify families are all valid
-      if (!scan.hasFamilies()) {
-        // Adding all families to scanner
-        for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
-          scan.addFamily(family);
-        }
-      } else {
-        for (byte[] family : scan.getFamilyMap().keySet()) {
-          checkFamily(family);
+    long nonceGroup, long nonce) throws IOException {
+    return TraceUtil.trace(() -> {
+      startRegionOperation(Operation.SCAN);
+      try {
+        // Verify families are all valid
+        if (!scan.hasFamilies()) {
+          // Adding all families to scanner
+          for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
+            scan.addFamily(family);
+          }
+        } else {
+          for (byte[] family : scan.getFamilyMap().keySet()) {
+            checkFamily(family);
+          }
         }
+        return instantiateRegionScanner(scan, additionalScanners, nonceGroup, nonce);
+      } finally {
+        closeRegionOperation(Operation.SCAN);
       }
-      return instantiateRegionScanner(scan, additionalScanners, nonceGroup, nonce);
-    } finally {
-      closeRegionOperation(Operation.SCAN);
-    }
+    }, () -> createRegionSpan("Region.getScanner"));
   }
 
   protected RegionScannerImpl instantiateRegionScanner(Scan scan,
@@ -3165,15 +3166,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   @Override
   public void delete(Delete delete) throws IOException {
-    checkReadOnly();
-    checkResources();
-    startRegionOperation(Operation.DELETE);
-    try {
-      // All edits for the given row (across all column families) must happen atomically.
-      mutate(delete);
-    } finally {
-      closeRegionOperation(Operation.DELETE);
-    }
+    TraceUtil.trace(() -> {
+      checkReadOnly();
+      checkResources();
+      startRegionOperation(Operation.DELETE);
+      try {
+        // All edits for the given row (across all column families) must happen atomically.
+        return mutate(delete);
+      } finally {
+        closeRegionOperation(Operation.DELETE);
+      }
+    }, () -> createRegionSpan("Region.delete"));
   }
 
   /**
@@ -3248,20 +3251,22 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   @Override
   public void put(Put put) throws IOException {
-    checkReadOnly();
+    TraceUtil.trace(() -> {
+      checkReadOnly();
 
-    // Do a rough check that we have resources to accept a write.  The check is
-    // 'rough' in that between the resource check and the call to obtain a
-    // read lock, resources may run out.  For now, the thought is that this
-    // will be extremely rare; we'll deal with it when it happens.
-    checkResources();
-    startRegionOperation(Operation.PUT);
-    try {
-      // All edits for the given row (across all column families) must happen atomically.
-      mutate(put);
-    } finally {
-      closeRegionOperation(Operation.PUT);
-    }
+      // Do a rough check that we have resources to accept a write. The check is
+      // 'rough' in that between the resource check and the call to obtain a
+      // read lock, resources may run out. For now, the thought is that this
+      // will be extremely rare; we'll deal with it when it happens.
+      checkResources();
+      startRegionOperation(Operation.PUT);
+      try {
+        // All edits for the given row (across all column families) must happen atomically.
+        return mutate(put);
+      } finally {
+        closeRegionOperation(Operation.PUT);
+      }
+    }, () -> createRegionSpan("Region.put"));
   }
 
   /**
@@ -3542,7 +3547,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         boolean throwException = false;
         try {
           // if atomic then get exclusive lock, else shared lock
-          rowLock = region.getRowLockInternal(mutation.getRow(), !isAtomic(), prevRowLock);
+          rowLock = region.getRowLock(mutation.getRow(), !isAtomic(), prevRowLock);
         } catch (TimeoutIOException | InterruptedIOException e) {
           // NOTE: We will retry when other exceptions, but we should stop if we receive
           // TimeoutIOException or InterruptedIOException as operation has timed out or
@@ -4326,7 +4331,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           this.checkAndPrepareMutation(cpMutation, timestamp);
 
           // Acquire row locks. If not, the whole batch will fail.
-          acquiredRowLocks.add(region.getRowLockInternal(cpMutation.getRow(), true, null));
+          acquiredRowLocks.add(region.getRowLock(cpMutation.getRow(), true, null));
 
           // Returned mutations from coprocessor correspond to the Mutation at index i. We can
           // directly add the cells from those mutations to the familyMaps of this mutation.
@@ -4497,7 +4502,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic) throws IOException {
-    return batchMutate(mutations, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE);
+    return TraceUtil.trace(
+      () -> batchMutate(mutations, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE),
+      () -> createRegionSpan("Region.batchMutate"));
   }
 
   public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId)
@@ -4779,6 +4786,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
   public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate, long nonceGroup,
     long nonce) throws IOException {
+    return TraceUtil.trace(() -> checkAndMutateInternal(checkAndMutate, nonceGroup, nonce),
+      () -> createRegionSpan("Region.checkAndMutate"));
+  }
+
+  private CheckAndMutateResult checkAndMutateInternal(CheckAndMutate checkAndMutate,
+    long nonceGroup, long nonce) throws IOException {
     byte[] row = checkAndMutate.getRow();
     Filter filter = null;
     byte[] family = null;
@@ -4827,7 +4840,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       }
       // Lock row - note that doBatchMutate will relock this row if called
       checkRow(row, "doCheckAndRowMutate");
-      RowLock rowLock = getRowLockInternal(get.getRow(), false, null);
+      RowLock rowLock = getRowLock(get.getRow(), false, null);
       try {
         if (this.getCoprocessorHost() != null) {
           CheckAndMutateResult result =
@@ -4837,7 +4850,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           }
         }
 
-        // NOTE: We used to wait here until mvcc caught up:  mvcc.await();
+        // NOTE: We used to wait here until mvcc caught up: mvcc.await();
         // Supposition is that now all changes are done under row locks, then when we go to read,
         // we'll get the latest on this row.
         boolean matches = false;
@@ -4885,7 +4898,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
             // And else 'delete' is not needed since it already does a second get, and sets the
             // timestamp from get (see prepareDeleteTimestamps).
           } else {
-            for (Mutation m: rowMutations.getMutations()) {
+            for (Mutation m : rowMutations.getMutations()) {
               if (m instanceof Put) {
                 updateCellTimestamps(m.getFamilyCellMap().values(), byteTs);
               }
@@ -4913,8 +4926,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
-  private void checkMutationType(final Mutation mutation)
-  throws DoNotRetryIOException {
+  private void checkMutationType(final Mutation mutation) throws DoNotRetryIOException {
     if (!(mutation instanceof Put) && !(mutation instanceof Delete) &&
       !(mutation instanceof Increment) && !(mutation instanceof Append)) {
       throw new org.apache.hadoop.hbase.DoNotRetryIOException(
@@ -6562,11 +6574,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   @Override
   public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
     checkRow(row, "row lock");
-    return getRowLockInternal(row, readLock, null);
+    return getRowLock(row, readLock, null);
   }
 
-  protected RowLock getRowLockInternal(byte[] row, boolean readLock, final RowLock prevRowLock)
-      throws IOException {
+  Span createRegionSpan(String name) {
+    return TraceUtil.createSpan(name).setAttribute(TraceUtil.REGION_NAMES_KEY,
+      Arrays.asList(getRegionInfo().getRegionNameAsString()));
+  }
+
+  // will be override in tests
+  protected RowLock getRowLockInternal(byte[] row, boolean readLock, RowLock prevRowLock)
+    throws IOException {
     // create an object to use a a key in the row lock map
     HashedBytes rowKey = new HashedBytes(row);
 
@@ -6574,9 +6592,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     RowLockImpl result = null;
 
     boolean success = false;
-    Span span = TraceUtil.getGlobalTracer().spanBuilder("HRegion.getRowLock").startSpan();
-    try (Scope scope = span.makeCurrent()) {
-      span.addEvent("Getting a " + (readLock ? "readLock" : "writeLock"));
+    try {
       // Keep trying until we have a lock or error out.
       // TODO: do we need to add a time component here?
       while (result == null) {
@@ -6613,7 +6629,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       }
 
       if (timeout <= 0 || !result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS)) {
-        span.addEvent("Failed to get row lock");
         String message = "Timed out waiting for lock for row: " + rowKey + " in region "
             + getRegionInfo().getEncodedName();
         if (reachDeadlineFirst) {
@@ -6631,7 +6646,6 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey,
           getRegionInfo().getRegionNameAsString());
       }
-      span.addEvent("Interrupted exception getting row lock");
       throw throwOnInterrupt(ie);
     } catch (Error error) {
       // The maximum lock count for read lock is 64K (hardcoded), when this maximum count
@@ -6640,17 +6654,22 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       LOG.warn("Error to get row lock for {}, in region {}, cause: {}", Bytes.toStringBinary(row),
         getRegionInfo().getRegionNameAsString(), error);
       IOException ioe = new IOException(error);
-      span.addEvent("Error getting row lock");
       throw ioe;
     } finally {
       // Clean up the counts just in case this was the thing keeping the context alive.
       if (!success && rowLockContext != null) {
         rowLockContext.cleanUp();
       }
-      span.end();
     }
   }
 
+  private RowLock getRowLock(byte[] row, boolean readLock, final RowLock prevRowLock)
+    throws IOException {
+    return TraceUtil.trace(() -> getRowLockInternal(row, readLock, prevRowLock),
+      () -> createRegionSpan("Region.getRowLock").setAttribute(TraceUtil.ROW_LOCK_READ_LOCK_KEY,
+        readLock));
+  }
+
   private void releaseRowLocks(List<RowLock> rowLocks) {
     if (rowLocks != null) {
       for (RowLock rowLock : rowLocks) {
@@ -7514,9 +7533,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   private List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
-      throws IOException {
+    throws IOException {
+    return TraceUtil.trace(() -> getInternal(get, withCoprocessor, nonceGroup, nonce),
+      () -> createRegionSpan("Region.get"));
+  }
+
+  private List<Cell> getInternal(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
+    throws IOException {
     List<Cell> results = new ArrayList<>();
-    long before =  EnvironmentEdgeManager.currentTime();
+    long before = EnvironmentEdgeManager.currentTime();
 
     // pre-get CP hook
     if (withCoprocessor && (coprocessorHost != null)) {
@@ -7613,7 +7638,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         RowLock prevRowLock = null;
         for (byte[] row : rowsToLock) {
           try {
-            RowLock rowLock = region.getRowLockInternal(row, false, prevRowLock); // write lock
+            RowLock rowLock = region.getRowLock(row, false, prevRowLock); // write lock
             if (rowLock != prevRowLock) {
               acquiredRowLocks.add(rowLock);
               prevRowLock = rowLock;
@@ -7881,15 +7906,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   public Result append(Append append, long nonceGroup, long nonce) throws IOException {
-    checkReadOnly();
-    checkResources();
-    startRegionOperation(Operation.APPEND);
-    try {
-      // All edits for the given row (across all column families) must happen atomically.
-      return mutate(append, true, nonceGroup, nonce).getResult();
-    } finally {
-      closeRegionOperation(Operation.APPEND);
-    }
+    return TraceUtil.trace(() -> {
+      checkReadOnly();
+      checkResources();
+      startRegionOperation(Operation.APPEND);
+      try {
+        // All edits for the given row (across all column families) must happen atomically.
+        return mutate(append, true, nonceGroup, nonce).getResult();
+      } finally {
+        closeRegionOperation(Operation.APPEND);
+      }
+    }, () -> createRegionSpan("Region.append"));
   }
 
   @Override
@@ -7898,15 +7925,17 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   }
 
   public Result increment(Increment increment, long nonceGroup, long nonce) throws IOException {
-    checkReadOnly();
-    checkResources();
-    startRegionOperation(Operation.INCREMENT);
-    try {
-      // All edits for the given row (across all column families) must happen atomically.
-      return mutate(increment, true, nonceGroup, nonce).getResult();
-    } finally {
-      closeRegionOperation(Operation.INCREMENT);
-    }
+    return TraceUtil.trace(() -> {
+      checkReadOnly();
+      checkResources();
+      startRegionOperation(Operation.INCREMENT);
+      try {
+        // All edits for the given row (across all column families) must happen atomically.
+        return mutate(increment, true, nonceGroup, nonce).getResult();
+      } finally {
+        closeRegionOperation(Operation.INCREMENT);
+      }
+    }, () -> createRegionSpan("Region.increment"));
   }
 
   private WriteEntry doWALAppend(WALEdit walEdit, Durability durability, List<UUID> clusterIds,
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java
index 612bb5c..b2e793f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionScannerImpl.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.regionserver.Region.Operation;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
+import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
@@ -254,37 +255,39 @@ class RegionScannerImpl implements RegionScanner, Shipper, RpcCallback {
 
   @Override
   public boolean nextRaw(List<Cell> outResults, ScannerContext scannerContext) throws IOException {
-    if (storeHeap == null) {
-      // scanner is closed
-      throw new UnknownScannerException("Scanner was closed");
-    }
-    boolean moreValues = false;
-    if (outResults.isEmpty()) {
-      // Usually outResults is empty. This is true when next is called
-      // to handle scan or get operation.
-      moreValues = nextInternal(outResults, scannerContext);
-    } else {
-      List<Cell> tmpList = new ArrayList<>();
-      moreValues = nextInternal(tmpList, scannerContext);
-      outResults.addAll(tmpList);
-    }
+    return TraceUtil.trace(() -> {
+      if (storeHeap == null) {
+        // scanner is closed
+        throw new UnknownScannerException("Scanner was closed");
+      }
+      boolean moreValues = false;
+      if (outResults.isEmpty()) {
+        // Usually outResults is empty. This is true when next is called
+        // to handle scan or get operation.
+        moreValues = nextInternal(outResults, scannerContext);
+      } else {
+        List<Cell> tmpList = new ArrayList<>();
+        moreValues = nextInternal(tmpList, scannerContext);
+        outResults.addAll(tmpList);
+      }
 
-    region.addReadRequestsCount(1);
-    if (region.getMetrics() != null) {
-      region.getMetrics().updateReadRequestCount();
-    }
+      region.addReadRequestsCount(1);
+      if (region.getMetrics() != null) {
+        region.getMetrics().updateReadRequestCount();
+      }
 
-    // If the size limit was reached it means a partial Result is being returned. Returning a
-    // partial Result means that we should not reset the filters; filters should only be reset in
-    // between rows
-    if (!scannerContext.mayHaveMoreCellsInRow()) {
-      resetFilters();
-    }
+      // If the size limit was reached it means a partial Result is being returned. Returning a
+      // partial Result means that we should not reset the filters; filters should only be reset in
+      // between rows
+      if (!scannerContext.mayHaveMoreCellsInRow()) {
+        resetFilters();
+      }
 
-    if (isFilterDoneInternal()) {
-      moreValues = false;
-    }
-    return moreValues;
+      if (isFilterDoneInternal()) {
+        moreValues = false;
+      }
+      return moreValues;
+    }, () -> region.createRegionSpan("RegionScanner.next"));
   }
 
   /**
@@ -723,8 +726,9 @@ class RegionScannerImpl implements RegionScanner, Shipper, RpcCallback {
     return c > 0 || (c == 0 && !includeStopRow);
   }
 
-  @Override
-  public synchronized void close() {
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "IS2_INCONSISTENT_SYNC",
+    justification = "this method is only called inside close which is synchronized")
+  private void closeInternal() {
     if (storeHeap != null) {
       storeHeap.close();
       storeHeap = null;
@@ -739,23 +743,30 @@ class RegionScannerImpl implements RegionScanner, Shipper, RpcCallback {
   }
 
   @Override
+  public synchronized void close() {
+    TraceUtil.trace(this::closeInternal, () -> region.createRegionSpan("RegionScanner.close"));
+  }
+
+  @Override
   public synchronized boolean reseek(byte[] row) throws IOException {
-    if (row == null) {
-      throw new IllegalArgumentException("Row cannot be null.");
-    }
-    boolean result = false;
-    region.startRegionOperation();
-    Cell kv = PrivateCellUtil.createFirstOnRow(row, 0, (short) row.length);
-    try {
-      // use request seek to make use of the lazy seek option. See HBASE-5520
-      result = this.storeHeap.requestSeek(kv, true, true);
-      if (this.joinedHeap != null) {
-        result = this.joinedHeap.requestSeek(kv, true, true) || result;
+    return TraceUtil.trace(() -> {
+      if (row == null) {
+        throw new IllegalArgumentException("Row cannot be null.");
       }
-    } finally {
-      region.closeRegionOperation();
-    }
-    return result;
+      boolean result = false;
+      region.startRegionOperation();
+      Cell kv = PrivateCellUtil.createFirstOnRow(row, 0, (short) row.length);
+      try {
+        // use request seek to make use of the lazy seek option. See HBASE-5520
+        result = this.storeHeap.requestSeek(kv, true, true);
+        if (this.joinedHeap != null) {
+          result = this.joinedHeap.requestSeek(kv, true, true) || result;
+        }
+      } finally {
+        region.closeRegionOperation();
+      }
+      return result;
+    }, () -> region.createRegionSpan("RegionScanner.reseek"));
   }
 
   @Override
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
index 0707a53..04aa3df 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
@@ -719,7 +719,7 @@ public class TestAtomicOperation {
     }
 
     @Override
-    public RowLock getRowLockInternal(final byte[] row, boolean readLock,
+    protected RowLock getRowLockInternal(final byte[] row, boolean readLock,
         final RowLock prevRowlock) throws IOException {
       if (testStep == TestStep.CHECKANDPUT_STARTED) {
         latch.countDown();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
new file mode 100644
index 0000000..3a772a2
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
@@ -0,0 +1,187 @@
+/**
+ * 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.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import static org.junit.Assert.assertTrue;
+
+import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNameTestRule;
+import org.apache.hadoop.hbase.client.Append;
+import org.apache.hadoop.hbase.client.CheckAndMutate;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Increment;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.trace.TraceUtil;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WAL;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
+
+@Category({ RegionServerTests.class, MediumTests.class })
+public class TestHRegionTracing {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestHRegionTracing.class);
+
+  private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  private static byte[] FAMILY = Bytes.toBytes("family");
+
+  private static byte[] QUALIFIER = Bytes.toBytes("qual");
+
+  private static byte[] ROW = Bytes.toBytes("row");
+
+  private static byte[] VALUE = Bytes.toBytes("value");
+
+  @Rule
+  public final OpenTelemetryRule traceRule = OpenTelemetryRule.create();
+
+  @Rule
+  public final TableNameTestRule tableNameRule = new TableNameTestRule();
+
+  private static WAL WAL;
+
+  private HRegion region;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws IOException {
+    WAL = HBaseTestingUtility.createWal(UTIL.getConfiguration(), UTIL.getDataTestDir(), null);
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws IOException {
+    Closeables.close(WAL, true);
+    UTIL.cleanupTestDir();
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    TableName tableName = tableNameRule.getTableName();
+    TableDescriptor desc = TableDescriptorBuilder.newBuilder(tableName)
+      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
+    RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
+    ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null,
+      MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);
+    region = HRegion.createHRegion(info, UTIL.getDataTestDir(), UTIL.getConfiguration(), desc, WAL);
+    region = UTIL.createLocalHRegion(info, desc);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    if (region != null) {
+      region.close();
+    }
+  }
+
+  private void assertSpan(String spanName) {
+    assertTrue(traceRule.getSpans().stream().anyMatch(span -> {
+      if (!span.getName().equals(spanName)) {
+        return false;
+      }
+      List<String> regionNames = span.getAttributes().get(TraceUtil.REGION_NAMES_KEY);
+      return regionNames != null && regionNames.size() == 1 &&
+        regionNames.get(0).equals(region.getRegionInfo().getRegionNameAsString());
+    }));
+  }
+
+  @Test
+  public void testGet() throws IOException {
+    region.get(new Get(ROW));
+    assertSpan("Region.get");
+  }
+
+  @Test
+  public void testPut() throws IOException {
+    region.put(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE));
+    assertSpan("Region.put");
+    assertSpan("Region.getRowLock");
+  }
+
+  @Test
+  public void testDelete() throws IOException {
+    region.delete(new Delete(ROW).addColumn(FAMILY, QUALIFIER));
+    assertSpan("Region.delete");
+    assertSpan("Region.getRowLock");
+  }
+
+  @Test
+  public void testAppend() throws IOException {
+    region.append(new Append(ROW).addColumn(FAMILY, QUALIFIER, VALUE));
+    assertSpan("Region.append");
+    assertSpan("Region.getRowLock");
+  }
+
+  @Test
+  public void testIncrement() throws IOException {
+    region.increment(new Increment(ROW).addColumn(FAMILY, QUALIFIER, 1));
+    assertSpan("Region.increment");
+    assertSpan("Region.getRowLock");
+  }
+
+  @Test
+  public void testBatchMutate() throws IOException {
+    region.batchMutate(new Mutation[] { new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE) });
+    assertSpan("Region.batchMutate");
+    assertSpan("Region.getRowLock");
+  }
+
+  @Test
+  public void testCheckAndMutate() throws IOException {
+    region.checkAndMutate(CheckAndMutate.newBuilder(ROW).ifNotExists(FAMILY, QUALIFIER)
+      .build(new Put(ROW).addColumn(FAMILY, QUALIFIER, VALUE)));
+    assertSpan("Region.checkAndMutate");
+    assertSpan("Region.getRowLock");
+  }
+
+  @Test
+  public void testScanner() throws IOException {
+    try (RegionScanner scanner = region.getScanner(new Scan())) {
+      scanner.reseek(ROW);
+      scanner.next(new ArrayList<>());
+    }
+    assertSpan("Region.getScanner");
+    assertSpan("RegionScanner.reseek");
+    assertSpan("RegionScanner.next");
+    assertSpan("RegionScanner.close");
+  }
+}

[hbase] 08/18: HBASE-26131 Backport HBASE-25484 "Add trace support for WAL sync" to branch-2 (#3597)

Posted by ta...@apache.org.
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 9cce94a2d05db892570b6a727d8e3fee5a7ac5ed
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Thu Aug 19 09:15:05 2021 -0700

    HBASE-26131 Backport HBASE-25484 "Add trace support for WAL sync" to branch-2 (#3597)
    
    8/17 commits of HBASE-22120, original commit 2be2c63f0d3917a243b74af9754cbfc805b858d1
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by:  Peter Somogyi <ps...@apache.org>
---
 .../org/apache/hadoop/hbase/trace/TraceUtil.java   |   2 +
 .../hbase/regionserver/wal/AbstractFSWAL.java      |  77 +++++++++++-----
 .../hadoop/hbase/regionserver/wal/AsyncFSWAL.java  | 101 +++++++--------------
 .../hadoop/hbase/regionserver/wal/FSHLog.java      |  38 +-------
 .../hadoop/hbase/regionserver/TestHRegion.java     |   4 +-
 .../regionserver/wal/TestAsyncFSWALDurability.java |   8 +-
 .../regionserver/wal/TestFSHLogDurability.java     |   8 +-
 .../org/apache/hadoop/hbase/wal/FaultyFSLog.java   |  10 +-
 8 files changed, 104 insertions(+), 144 deletions(-)

diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
index ea16df1..886a4a9 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
@@ -62,6 +62,8 @@ public final class TraceUtil {
   public static final AttributeKey<Boolean> ROW_LOCK_READ_LOCK_KEY =
     AttributeKey.booleanKey("db.hbase.rowlock.readlock");
 
+  public static final AttributeKey<String> WAL_IMPL = AttributeKey.stringKey("db.hbase.wal.impl");
+
   private TraceUtil() {
   }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
index 5c247d1..c39e10d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java
@@ -26,7 +26,6 @@ import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.c
 
 import com.lmax.disruptor.RingBuffer;
 import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
@@ -549,6 +548,35 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     return rollWriter(false);
   }
 
+  @Override
+  public final void sync() throws IOException {
+    sync(useHsync);
+  }
+
+  @Override
+  public final void sync(long txid) throws IOException {
+    sync(txid, useHsync);
+  }
+
+  @Override
+  public final void sync(boolean forceSync) throws IOException {
+    TraceUtil.trace(() -> {
+      doSync(forceSync);
+      return null;
+    }, () -> createSpan("WAL.sync"));
+  }
+
+  @Override
+  public final void sync(long txid, boolean forceSync) throws IOException {
+    TraceUtil.trace(() -> {
+      doSync(txid, forceSync);
+      return null;
+    }, () -> createSpan("WAL.sync"));
+  }
+
+  protected abstract void doSync(boolean forceSync) throws IOException;
+
+  protected abstract void doSync(long txid, boolean forceSync) throws IOException;
   /**
    * This is a convenience method that computes a new filename with a given file-number.
    * @param filenum to use
@@ -650,7 +678,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
         this.sequenceIdAccounting.findLower(firstWALEntry.getValue().encodedName2HighestSequenceId);
     }
     if (regions != null) {
-      List<String> listForPrint = new ArrayList();
+      List<String> listForPrint = new ArrayList<>();
       for (Map.Entry<byte[], List<byte[]>> r : regions.entrySet()) {
         StringBuilder families = new StringBuilder();
         for (int i = 0; i < r.getValue().size(); i++) {
@@ -768,6 +796,10 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     }
   }
 
+  private Span createSpan(String name) {
+    return TraceUtil.createSpan(name).setAttribute(TraceUtil.WAL_IMPL, implClassName);
+  }
+
   /**
    * Cleans up current writer closing it and then puts in place the passed in {@code nextWriter}.
    * <p/>
@@ -785,13 +817,10 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
    * @throws IOException if there is a problem flushing or closing the underlying FS
    */
   Path replaceWriter(Path oldPath, Path newPath, W nextWriter) throws IOException {
-    Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHFile.replaceWriter").startSpan();
-    try (Scope scope = span.makeCurrent()) {
+    return TraceUtil.trace(() -> {
       doReplaceWriter(oldPath, newPath, nextWriter);
       return newPath;
-    } finally {
-      span.end();
-    }
+    }, () -> createSpan("WAL.replaceWriter"));
   }
 
   protected final void blockOnSync(SyncFuture syncFuture) throws IOException {
@@ -825,8 +854,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     return ioe;
   }
 
-  @Override
-  public Map<byte[], List<byte[]>> rollWriter(boolean force) throws IOException {
+  private Map<byte[], List<byte[]>> rollWriterInternal(boolean force) throws IOException {
     rollWriterLock.lock();
     try {
       // Return if nothing to flush.
@@ -838,8 +866,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
         LOG.debug("WAL closed. Skipping rolling of writer");
         return regionsToFlush;
       }
-      Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHLog.rollWriter").startSpan();
-      try (Scope scope = span.makeCurrent()) {
+      try {
         Path oldPath = getOldPath();
         Path newPath = getNewPath();
         // Any exception from here on is catastrophic, non-recoverable so we currently abort.
@@ -864,10 +891,8 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
         // If the underlying FileSystem can't do what we ask, treat as IO failure so
         // we'll abort.
         throw new IOException(
-            "Underlying FileSystem can't meet stream requirements. See RS log " + "for details.",
-            exception);
-      } finally {
-        span.end();
+          "Underlying FileSystem can't meet stream requirements. See RS log " + "for details.",
+          exception);
       }
       return regionsToFlush;
     } finally {
@@ -875,6 +900,11 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     }
   }
 
+  @Override
+  public Map<byte[], List<byte[]>> rollWriter(boolean force) throws IOException {
+    return TraceUtil.trace(() -> rollWriterInternal(force), () -> createSpan("WAL.rollWriter"));
+  }
+
   // public only until class moves to o.a.h.h.wal
   /** @return the size of log files in use */
   public long getLogFileSize() {
@@ -1059,7 +1089,6 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
           .append(TimeUnit.NANOSECONDS.toMillis(timeInNanos))
           .append(" ms, current pipeline: ")
           .append(Arrays.toString(getPipeline())).toString();
-      Span.current().addEvent(msg);
       LOG.info(msg);
       // A single sync took too long.
       // Elsewhere in checkSlowSync, called from checkLogRoll, we will look at cumulative
@@ -1082,8 +1111,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
   }
 
   protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKeyImpl key,
-    WALEdit edits, boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer)
-    throws IOException {
+    WALEdit edits, boolean inMemstore, RingBuffer<RingBufferTruck> ringBuffer) throws IOException {
     if (this.closed) {
       throw new IOException(
         "Cannot append; log is closed, regionName = " + hri.getRegionNameAsString());
@@ -1095,14 +1123,12 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
     long txid = txidHolder.longValue();
     ServerCall<?> rpcCall = RpcServer.getCurrentCall().filter(c -> c instanceof ServerCall)
       .filter(c -> c.getCellScanner() != null).map(c -> (ServerCall) c).orElse(null);
-    Span span = TraceUtil.getGlobalTracer().spanBuilder(implClassName + ".append").startSpan();
-    try (Scope scope = span.makeCurrent()) {
+    try {
       FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore, rpcCall);
       entry.stampRegionSequenceId(we);
       ringBuffer.get(txid).load(entry);
     } finally {
       ringBuffer.publish(txid);
-      span.end();
     }
     return txid;
   }
@@ -1145,13 +1171,14 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
 
   @Override
   public long appendData(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException {
-    return append(info, key, edits, true);
+    return TraceUtil.trace(() -> append(info, key, edits, true),
+      () -> createSpan("WAL.appendData"));
   }
 
   @Override
-  public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits)
-    throws IOException {
-    return append(info, key, edits, false);
+  public long appendMarker(RegionInfo info, WALKeyImpl key, WALEdit edits) throws IOException {
+    return TraceUtil.trace(() -> append(info, key, edits, false),
+      () -> createSpan("WAL.appendMarker"));
   }
 
   /**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
index eef0575..54532b5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java
@@ -23,8 +23,6 @@ import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
 import com.lmax.disruptor.RingBuffer;
 import com.lmax.disruptor.Sequence;
 import com.lmax.disruptor.Sequencer;
-import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.lang.reflect.Field;
 import java.util.ArrayDeque;
@@ -52,7 +50,6 @@ import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.client.RegionInfo;
 import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput;
-import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
 import org.apache.hadoop.hbase.wal.WALEdit;
 import org.apache.hadoop.hbase.wal.WALKeyImpl;
@@ -349,7 +346,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
         break;
       }
     }
-    postSync(System.nanoTime() - startTimeNs, finishSync(true));
+    postSync(System.nanoTime() - startTimeNs, finishSync());
     if (trySetReadyForRolling()) {
       // we have just finished a roll, then do not need to check for log rolling, the writer will be
       // closed soon.
@@ -399,13 +396,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     }, consumeExecutor);
   }
 
-  private void addTimeAnnotation(SyncFuture future, String annotation) {
-    Span.current().addEvent(annotation);
-    // TODO handle htrace API change, see HBASE-18895
-    // future.setSpan(scope.getSpan());
-  }
-
-  private int finishSyncLowerThanTxid(long txid, boolean addSyncTrace) {
+  private int finishSyncLowerThanTxid(long txid) {
     int finished = 0;
     for (Iterator<SyncFuture> iter = syncFutures.iterator(); iter.hasNext();) {
       SyncFuture sync = iter.next();
@@ -413,9 +404,6 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
         markFutureDoneAndOffer(sync, txid, null);
         iter.remove();
         finished++;
-        if (addSyncTrace) {
-          addTimeAnnotation(sync, "writer synced");
-        }
       } else {
         break;
       }
@@ -424,7 +412,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   }
 
   // try advancing the highestSyncedTxid as much as possible
-  private int finishSync(boolean addSyncTrace) {
+  private int finishSync() {
     if (unackedAppends.isEmpty()) {
       // All outstanding appends have been acked.
       if (toWriteAppends.isEmpty()) {
@@ -432,10 +420,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
         long maxSyncTxid = highestSyncedTxid.get();
         for (SyncFuture sync : syncFutures) {
           maxSyncTxid = Math.max(maxSyncTxid, sync.getTxid());
-          markFutureDoneAndOffer(sync, maxSyncTxid, null);
-          if (addSyncTrace) {
-            addTimeAnnotation(sync, "writer synced");
-          }
+          sync.done(maxSyncTxid, null);
         }
         highestSyncedTxid.set(maxSyncTxid);
         int finished = syncFutures.size();
@@ -449,7 +434,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
         assert lowestUnprocessedAppendTxid > highestProcessedAppendTxid;
         long doneTxid = lowestUnprocessedAppendTxid - 1;
         highestSyncedTxid.set(doneTxid);
-        return finishSyncLowerThanTxid(doneTxid, addSyncTrace);
+        return finishSyncLowerThanTxid(doneTxid);
       }
     } else {
       // There are still unacked appends. So let's move the highestSyncedTxid to the txid of the
@@ -457,7 +442,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
       long lowestUnackedAppendTxid = unackedAppends.peek().getTxid();
       long doneTxid = Math.max(lowestUnackedAppendTxid - 1, highestSyncedTxid.get());
       highestSyncedTxid.set(doneTxid);
-      return finishSyncLowerThanTxid(doneTxid, addSyncTrace);
+      return finishSyncLowerThanTxid(doneTxid);
     }
   }
 
@@ -465,7 +450,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
     final AsyncWriter writer = this.writer;
     // maybe a sync request is not queued when we issue a sync, so check here to see if we could
     // finish some.
-    finishSync(false);
+    finishSync();
     long newHighestProcessedAppendTxid = -1L;
     for (Iterator<FSWALEntry> iter = toWriteAppends.iterator(); iter.hasNext();) {
       FSWALEntry entry = iter.next();
@@ -506,7 +491,7 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
       // stamped some region sequence id.
       if (unackedAppends.isEmpty()) {
         highestSyncedTxid.set(highestProcessedAppendTxid);
-        finishSync(false);
+        finishSync();
         trySetReadyForRolling();
       }
       return;
@@ -612,61 +597,41 @@ public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
   }
 
   @Override
-  public void sync() throws IOException {
-    sync(useHsync);
-  }
-
-  @Override
-  public void sync(long txid) throws IOException {
-    sync(txid, useHsync);
-  }
-
-  @Override
-  public void sync(boolean forceSync) throws IOException {
-    Span span = TraceUtil.getGlobalTracer().spanBuilder("AsyncFSWAL.sync").startSpan();
-    try (Scope scope = span.makeCurrent()) {
-      long txid = waitingConsumePayloads.next();
-      SyncFuture future;
-      try {
-        future = getSyncFuture(txid, forceSync);
-        RingBufferTruck truck = waitingConsumePayloads.get(txid);
-        truck.load(future);
-      } finally {
-        waitingConsumePayloads.publish(txid);
-      }
-      if (shouldScheduleConsumer()) {
-        consumeExecutor.execute(consumer);
-      }
-      blockOnSync(future);
+  protected void doSync(boolean forceSync) throws IOException {
+    long txid = waitingConsumePayloads.next();
+    SyncFuture future;
+    try {
+      future = getSyncFuture(txid, forceSync);
+      RingBufferTruck truck = waitingConsumePayloads.get(txid);
+      truck.load(future);
     } finally {
-      span.end();
+      waitingConsumePayloads.publish(txid);
+    }
+    if (shouldScheduleConsumer()) {
+      consumeExecutor.execute(consumer);
     }
+    blockOnSync(future);
   }
 
   @Override
-  public void sync(long txid, boolean forceSync) throws IOException {
+  protected void doSync(long txid, boolean forceSync) throws IOException {
     if (highestSyncedTxid.get() >= txid) {
       return;
     }
-    Span span = TraceUtil.getGlobalTracer().spanBuilder("AsyncFSWAL.sync").startSpan();
-    try (Scope scope = span.makeCurrent()) {
-      // here we do not use ring buffer sequence as txid
-      long sequence = waitingConsumePayloads.next();
-      SyncFuture future;
-      try {
-        future = getSyncFuture(txid, forceSync);
-        RingBufferTruck truck = waitingConsumePayloads.get(sequence);
-        truck.load(future);
-      } finally {
-        waitingConsumePayloads.publish(sequence);
-      }
-      if (shouldScheduleConsumer()) {
-        consumeExecutor.execute(consumer);
-      }
-      blockOnSync(future);
+    // here we do not use ring buffer sequence as txid
+    long sequence = waitingConsumePayloads.next();
+    SyncFuture future;
+    try {
+      future = getSyncFuture(txid, forceSync);
+      RingBufferTruck truck = waitingConsumePayloads.get(sequence);
+      truck.load(future);
     } finally {
-      span.end();
+      waitingConsumePayloads.publish(sequence);
+    }
+    if (shouldScheduleConsumer()) {
+      consumeExecutor.execute(consumer);
     }
+    blockOnSync(future);
   }
 
   @Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
index 762f1a1..8d61cf9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
@@ -30,7 +30,6 @@ import com.lmax.disruptor.TimeoutException;
 import com.lmax.disruptor.dsl.Disruptor;
 import com.lmax.disruptor.dsl.ProducerType;
 import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.context.Scope;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.Arrays;
@@ -49,7 +48,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.RegionInfo;
-import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.CommonFSUtils;
@@ -365,7 +363,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
           // use assert to make sure no change breaks the logic that
           // sequence and zigzagLatch will be set together
           assert sequence > 0L : "Failed to get sequence from ring buffer";
-          Span.current().addEvent("awaiting safepoint");
           syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer(sequence, false));
         }
       } catch (FailedSyncBeforeLogCloseException e) {
@@ -646,15 +643,11 @@ public class FSHLog extends AbstractFSWAL<Writer> {
           }
           // I got something. Lets run. Save off current sequence number in case it changes
           // while we run.
-          //TODO handle htrace API change, see HBASE-18895
-          //TraceScope scope = Trace.continueSpan(takeSyncFuture.getSpan());
           long start = System.nanoTime();
           Throwable lastException = null;
           try {
-            Span.current().addEvent("syncing writer");
             long unSyncedFlushSeq = highestUnsyncedTxid;
             writer.sync(sf.isForceSync());
-            Span.current().addEvent("writer synced");
             if (unSyncedFlushSeq > currentSequence) {
               currentSequence = unSyncedFlushSeq;
             }
@@ -666,9 +659,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
             LOG.warn("UNEXPECTED", e);
             lastException = e;
           } finally {
-            // reattach the span to the future before releasing.
-            //TODO handle htrace API change, see HBASE-18895
-            // takeSyncFuture.setSpan(scope.getSpan());
             // First release what we 'took' from the queue.
             syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, lastException);
             // Can we release other syncs?
@@ -793,7 +783,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   }
 
   // Sync all known transactions
-  private void publishSyncThenBlockOnCompletion(Scope scope, boolean forceSync) throws IOException {
+  private void publishSyncThenBlockOnCompletion(boolean forceSync) throws IOException {
     SyncFuture syncFuture = publishSyncOnRingBuffer(forceSync);
     blockOnSync(syncFuture);
   }
@@ -819,33 +809,17 @@ public class FSHLog extends AbstractFSWAL<Writer> {
   }
 
   @Override
-  public void sync() throws IOException {
-    sync(useHsync);
+  protected void doSync(boolean forceSync) throws IOException {
+    publishSyncThenBlockOnCompletion(forceSync);
   }
 
   @Override
-  public void sync(boolean forceSync) throws IOException {
-    Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHLog.sync").startSpan();
-    try (Scope scope = span.makeCurrent()) {
-      publishSyncThenBlockOnCompletion(scope, forceSync);
-    }
-  }
-
-  @Override
-  public void sync(long txid) throws IOException {
-    sync(txid, useHsync);
-  }
-
-  @Override
-  public void sync(long txid, boolean forceSync) throws IOException {
+  protected void doSync(long txid, boolean forceSync) throws IOException {
     if (this.highestSyncedTxid.get() >= txid) {
       // Already sync'd.
       return;
     }
-    Span span = TraceUtil.getGlobalTracer().spanBuilder("FSHLog.sync").startSpan();
-    try (Scope scope = span.makeCurrent()) {
-      publishSyncThenBlockOnCompletion(scope, forceSync);
-    }
+    publishSyncThenBlockOnCompletion(forceSync);
   }
 
   boolean isLowReplicationRollEnabled() {
@@ -1066,8 +1040,6 @@ public class FSHLog extends AbstractFSWAL<Writer> {
           }
         } else if (truck.type() == RingBufferTruck.Type.APPEND) {
           FSWALEntry entry = truck.unloadAppend();
-          //TODO handle htrace API change, see HBASE-18895
-          //TraceScope scope = Trace.continueSpan(entry.detachSpan());
           try {
             if (this.exception != null) {
               // Return to keep processing events coming off the ringbuffer
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 9763841..b4e0110 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -351,9 +351,9 @@ public class TestHRegion {
       }
 
       @Override
-      public void sync(long txid) throws IOException {
+      protected void doSync(long txid, boolean forceSync) throws IOException {
         storeFlushCtx.prepare();
-        super.sync(txid);
+        super.doSync(txid, forceSync);
       }
     }
 
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWALDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWALDurability.java
index a482d93..a2ac337 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWALDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestAsyncFSWALDurability.java
@@ -128,15 +128,15 @@ class CustomAsyncFSWAL extends AsyncFSWAL {
   }
 
   @Override
-  public void sync(boolean forceSync) throws IOException {
+  protected void doSync(boolean forceSync) throws IOException {
     syncFlag = forceSync;
-    super.sync(forceSync);
+    super.doSync(forceSync);
   }
 
   @Override
-  public void sync(long txid, boolean forceSync) throws IOException {
+  protected void doSync(long txid, boolean forceSync) throws IOException {
     syncFlag = forceSync;
-    super.sync(txid, forceSync);
+    super.doSync(txid, forceSync);
   }
 
   void resetSyncFlag() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLogDurability.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLogDurability.java
index 3c25044..78531f3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLogDurability.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLogDurability.java
@@ -103,15 +103,15 @@ class CustomFSHLog extends FSHLog {
   }
 
   @Override
-  public void sync(boolean forceSync) throws IOException {
+  protected void doSync(boolean forceSync) throws IOException {
     syncFlag = forceSync;
-    super.sync(forceSync);
+    super.doSync(forceSync);
   }
 
   @Override
-  public void sync(long txid, boolean forceSync) throws IOException {
+  protected void doSync(long txid, boolean forceSync) throws IOException {
     syncFlag = forceSync;
-    super.sync(txid, forceSync);
+    super.doSync(txid, forceSync);
   }
 
   void resetSyncFlag() {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
index f58d364..3afafa1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
@@ -20,7 +20,6 @@
 package org.apache.hadoop.hbase.wal;
 
 import java.io.IOException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -50,16 +49,11 @@ public class FaultyFSLog extends FSHLog {
   }
 
   @Override
-  public void sync(long txid) throws IOException {
-    sync(txid, false);
-  }
-
-  @Override
-  public void sync(long txid, boolean forceSync) throws IOException {
+  protected void doSync(long txid, boolean forceSync) throws IOException {
     if (this.ft == FailureType.SYNC) {
       throw new IOException("sync");
     }
-    super.sync(txid, forceSync);
+    super.doSync(txid, forceSync);
   }
 
   @Override

[hbase] 10/18: HBASE-26133 Backport HBASE-25591 "Upgrade opentelemetry to 0.17.1" to branch-2 (#3608)

Posted by ta...@apache.org.
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 d0a53e3f297cc7060782f94e02c86c10d6114af7
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Mon Aug 23 08:52:07 2021 -0700

    HBASE-26133 Backport HBASE-25591 "Upgrade opentelemetry to 0.17.1" to branch-2 (#3608)
    
    10/17 commits of HBASE-22120, original commit f6ff519dd0c7fe0e3ae3c175eefee27a26a065a4
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 .../hadoop/hbase/client/TestAsyncTableTracing.java |  6 +++---
 hbase-common/pom.xml                               |  4 ++++
 .../org/apache/hadoop/hbase/trace/TraceUtil.java   | 25 +++++++++++-----------
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java     |  3 +--
 .../hadoop/hbase/ipc/ServerRpcConnection.java      |  4 ++--
 .../apache/hadoop/hbase/ipc/AbstractTestIPC.java   | 14 ++++++------
 .../hbase/regionserver/TestHRegionTracing.java     | 15 ++++++-------
 pom.xml                                            | 18 +++++++---------
 8 files changed, 44 insertions(+), 45 deletions(-)

diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java
index 07cdf0e..a3ad9df 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableTracing.java
@@ -24,7 +24,7 @@ import static org.mockito.ArgumentMatchers.anyLong;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 
-import io.opentelemetry.api.trace.Span.Kind;
+import io.opentelemetry.api.trace.SpanKind;
 import io.opentelemetry.api.trace.StatusCode;
 import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule;
 import io.opentelemetry.sdk.trace.data.SpanData;
@@ -222,7 +222,7 @@ public class TestAsyncTableTracing {
     Waiter.waitFor(CONF, 1000,
       () -> traceRule.getSpans().stream()
         .anyMatch(span -> span.getName().equals("AsyncTable." + methodName) &&
-          span.getKind() == Kind.INTERNAL && span.hasEnded()));
+          span.getKind() == SpanKind.INTERNAL && span.hasEnded()));
     SpanData data = traceRule.getSpans().stream()
       .filter(s -> s.getName().equals("AsyncTable." + methodName)).findFirst().get();
     assertEquals(StatusCode.OK, data.getStatus().getStatusCode());
@@ -409,7 +409,7 @@ public class TestAsyncTableTracing {
     Waiter.waitFor(CONF, 1000,
       () -> traceRule.getSpans().stream()
         .anyMatch(span -> span.getName().equals("AsyncConnection.close") &&
-          span.getKind() == Kind.INTERNAL && span.hasEnded()));
+          span.getKind() == SpanKind.INTERNAL && span.hasEnded()));
     SpanData data = traceRule.getSpans().stream()
       .filter(s -> s.getName().equals("AsyncConnection.close")).findFirst().get();
     assertEquals(StatusCode.OK, data.getStatus().getStatusCode());
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index 62661b8..2212fd1 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -196,6 +196,10 @@
       <artifactId>opentelemetry-api</artifactId>
     </dependency>
     <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-semconv</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-crypto</artifactId>
     </dependency>
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
index 2a6b6b6..43c2e04 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
@@ -20,12 +20,12 @@ package org.apache.hadoop.hbase.trace;
 import io.opentelemetry.api.GlobalOpenTelemetry;
 import io.opentelemetry.api.common.AttributeKey;
 import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.api.trace.Span.Kind;
+import io.opentelemetry.api.trace.SpanKind;
 import io.opentelemetry.api.trace.StatusCode;
 import io.opentelemetry.api.trace.Tracer;
-import io.opentelemetry.api.trace.attributes.SemanticAttributes;
 import io.opentelemetry.context.Context;
 import io.opentelemetry.context.Scope;
+import io.opentelemetry.semconv.trace.attributes.SemanticAttributes;
 import java.io.IOException;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
@@ -72,14 +72,14 @@ public final class TraceUtil {
   }
 
   /**
-   * Create a {@link Kind#INTERNAL} span.
+   * Create a {@link SpanKind#INTERNAL} span.
    */
   public static Span createSpan(String name) {
-    return createSpan(name, Kind.INTERNAL);
+    return createSpan(name, SpanKind.INTERNAL);
   }
 
   /**
-   * Create a {@link Kind#INTERNAL} span and set table related attributes.
+   * Create a {@link SpanKind#INTERNAL} span and set table related attributes.
    */
   public static Span createTableSpan(String spanName, TableName tableName) {
     return createSpan(spanName).setAttribute(NAMESPACE_KEY, tableName.getNamespaceAsString())
@@ -88,28 +88,29 @@ public final class TraceUtil {
 
   /**
    * Create a span with the given {@code kind}. Notice that, OpenTelemetry only expects one
-   * {@link Kind#CLIENT} span and one {@link Kind#SERVER} span for a traced request, so use this
-   * with caution when you want to create spans with kind other than {@link Kind#INTERNAL}.
+   * {@link SpanKind#CLIENT} span and one {@link SpanKind#SERVER} span for a traced request, so use
+   * this with caution when you want to create spans with kind other than {@link SpanKind#INTERNAL}.
    */
-  private static Span createSpan(String name, Kind kind) {
+  private static Span createSpan(String name, SpanKind kind) {
     return getGlobalTracer().spanBuilder(name).setSpanKind(kind).startSpan();
   }
 
   /**
    * Create a span which parent is from remote, i.e, passed through rpc.
    * </p>
-   * We will set the kind of the returned span to {@link Kind#SERVER}, as this should be the top
+   * We will set the kind of the returned span to {@link SpanKind#SERVER}, as this should be the top
    * most span at server side.
    */
   public static Span createRemoteSpan(String name, Context ctx) {
-    return getGlobalTracer().spanBuilder(name).setParent(ctx).setSpanKind(Kind.SERVER).startSpan();
+    return getGlobalTracer().spanBuilder(name).setParent(ctx).setSpanKind(SpanKind.SERVER)
+      .startSpan();
   }
 
   /**
-   * Create a span with {@link Kind#CLIENT}.
+   * Create a span with {@link SpanKind#CLIENT}.
    */
   public static Span createClientSpan(String name) {
-    return createSpan(name, Kind.CLIENT);
+    return createSpan(name, SpanKind.CLIENT);
   }
 
   /**
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
index 5e390d2..13e0f0a 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableDescriptor;
 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
@@ -377,7 +376,7 @@ public class IntegrationTestMTTR {
     public void addResult(long time, Span span) {
       stats.addValue(TimeUnit.MILLISECONDS.convert(time, TimeUnit.NANOSECONDS));
       if (TimeUnit.SECONDS.convert(time, TimeUnit.NANOSECONDS) >= 1) {
-        traces.add(span.getSpanContext().getTraceIdAsHexString());
+        traces.add(span.getSpanContext().getTraceId());
       }
     }
 
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
index 823005b..208ec8b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcConnection.java
@@ -23,7 +23,7 @@ import io.opentelemetry.api.GlobalOpenTelemetry;
 import io.opentelemetry.api.trace.Span;
 import io.opentelemetry.context.Context;
 import io.opentelemetry.context.Scope;
-import io.opentelemetry.context.propagation.TextMapPropagator;
+import io.opentelemetry.context.propagation.TextMapGetter;
 import java.io.ByteArrayInputStream;
 import java.io.Closeable;
 import java.io.DataOutputStream;
@@ -615,7 +615,7 @@ abstract class ServerRpcConnection implements Closeable {
     ProtobufUtil.mergeFrom(builder, cis, headerSize);
     RequestHeader header = (RequestHeader) builder.build();
     offset += headerSize;
-    TextMapPropagator.Getter<RPCTInfo> getter = new TextMapPropagator.Getter<RPCTInfo>() {
+    TextMapGetter<RPCTInfo> getter = new TextMapGetter<RPCTInfo>() {
 
       @Override
       public Iterable<String> keys(RPCTInfo carrier) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index c4951b2..ce75343 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -34,7 +34,7 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.verify;
 import static org.mockito.internal.verification.VerificationModeFactory.times;
 
-import io.opentelemetry.api.trace.Span.Kind;
+import io.opentelemetry.api.trace.SpanKind;
 import io.opentelemetry.api.trace.StatusCode;
 import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule;
 import io.opentelemetry.sdk.trace.data.SpanData;
@@ -457,7 +457,7 @@ public abstract class AbstractTestIPC {
   }
 
   private void assertRpcAttribute(SpanData data, String methodName, InetSocketAddress addr,
-    Kind kind) {
+    SpanKind kind) {
     assertEquals(SERVICE.getDescriptorForType().getName(),
       data.getAttributes().get(TraceUtil.RPC_SERVICE_KEY));
     assertEquals(methodName, data.getAttributes().get(TraceUtil.RPC_METHOD_KEY));
@@ -471,7 +471,7 @@ public abstract class AbstractTestIPC {
   private void assertRemoteSpan() {
     SpanData data = waitSpan("RpcServer.process");
     assertTrue(data.getParentSpanContext().isRemote());
-    assertEquals(Kind.SERVER, data.getKind());
+    assertEquals(SpanKind.SERVER, data.getKind());
   }
 
   @Test
@@ -484,8 +484,8 @@ public abstract class AbstractTestIPC {
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
       stub.pause(null, PauseRequestProto.newBuilder().setMs(100).build());
       assertRpcAttribute(waitSpan("RpcClient.callMethod"), "pause", rpcServer.getListenerAddress(),
-        Kind.CLIENT);
-      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "pause", null, Kind.INTERNAL);
+        SpanKind.CLIENT);
+      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "pause", null, SpanKind.INTERNAL);
       assertRemoteSpan();
       assertSameTraceId();
       for (SpanData data : traceRule.getSpans()) {
@@ -499,8 +499,8 @@ public abstract class AbstractTestIPC {
       assertThrows(ServiceException.class,
         () -> stub.error(null, EmptyRequestProto.getDefaultInstance()));
       assertRpcAttribute(waitSpan("RpcClient.callMethod"), "error", rpcServer.getListenerAddress(),
-        Kind.CLIENT);
-      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "error", null, Kind.INTERNAL);
+        SpanKind.CLIENT);
+      assertRpcAttribute(waitSpan("RpcServer.callMethod"), "error", null, SpanKind.INTERNAL);
       assertRemoteSpan();
       assertSameTraceId();
       for (SpanData data : traceRule.getSpans()) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
index 3a772a2..17c3eb4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
@@ -23,6 +23,7 @@ import io.opentelemetry.sdk.testing.junit4.OpenTelemetryRule;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.TableName;
@@ -48,7 +49,6 @@ import org.apache.hadoop.hbase.wal.WAL;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
@@ -79,18 +79,12 @@ public class TestHRegionTracing {
   @Rule
   public final TableNameTestRule tableNameRule = new TableNameTestRule();
 
-  private static WAL WAL;
+  private WAL wal;
 
   private HRegion region;
 
-  @BeforeClass
-  public static void setUpBeforeClass() throws IOException {
-    WAL = HBaseTestingUtility.createWal(UTIL.getConfiguration(), UTIL.getDataTestDir(), null);
-  }
-
   @AfterClass
   public static void tearDownAfterClass() throws IOException {
-    Closeables.close(WAL, true);
     UTIL.cleanupTestDir();
   }
 
@@ -102,7 +96,9 @@ public class TestHRegionTracing {
     RegionInfo info = RegionInfoBuilder.newBuilder(tableName).build();
     ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false, 0, 0, 0, null,
       MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);
-    region = HRegion.createHRegion(info, UTIL.getDataTestDir(), UTIL.getConfiguration(), desc, WAL);
+    wal = HBaseTestingUtility.createWal(UTIL.getConfiguration(),
+      new Path(UTIL.getDataTestDir(), tableName.getNameAsString()), null);
+    region = HRegion.createHRegion(info, UTIL.getDataTestDir(), UTIL.getConfiguration(), desc, wal);
     region = UTIL.createLocalHRegion(info, desc);
   }
 
@@ -111,6 +107,7 @@ public class TestHRegionTracing {
     if (region != null) {
       region.close();
     }
+    Closeables.close(wal, true);
   }
 
   private void assertSpan(String spanName) {
diff --git a/pom.xml b/pom.xml
index 9fcf86b..6f04e4d 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1483,7 +1483,8 @@
     <jruby.version>9.2.13.0</jruby.version>
     <junit.version>4.13</junit.version>
     <hamcrest.version>1.3</hamcrest.version>
-    <opentelemetry.version>0.13.1</opentelemetry.version>
+    <opentelemetry.version>0.17.1</opentelemetry.version>
+    <opentelemetry-javaagent.version>0.17.0</opentelemetry-javaagent.version>
     <log4j.version>1.2.17</log4j.version>
     <mockito-core.version>2.28.2</mockito-core.version>
     <!--Internally we use a different version of protobuf. See hbase-protocol-shaded-->
@@ -2176,23 +2177,20 @@
       </dependency>
       <dependency>
         <groupId>io.opentelemetry</groupId>
-        <artifactId>opentelemetry-api</artifactId>
-        <version>${opentelemetry.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>io.opentelemetry</groupId>
-        <artifactId>opentelemetry-sdk</artifactId>
+        <artifactId>opentelemetry-bom</artifactId>
         <version>${opentelemetry.version}</version>
+        <type>pom</type>
+        <scope>import</scope>
       </dependency>
       <dependency>
         <groupId>io.opentelemetry</groupId>
-        <artifactId>opentelemetry-sdk-testing</artifactId>
-        <version>${opentelemetry.version}</version>
+        <artifactId>opentelemetry-semconv</artifactId>
+        <version>${opentelemetry.version}-alpha</version>
       </dependency>
       <dependency>
         <groupId>io.opentelemetry.javaagent</groupId>
         <artifactId>opentelemetry-javaagent</artifactId>
-        <version>${opentelemetry.version}</version>
+        <version>${opentelemetry-javaagent.version}</version>
         <classifier>all</classifier>
       </dependency>
       <dependency>

[hbase] 15/18: HBASE-26138 Backport HBASE-25733 "Upgrade opentelemetry to 1.0.1" to branch-2 (#3628)

Posted by ta...@apache.org.
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 a746e19c3be6b69ce3a7f661a6b83cea05215daa
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Wed Aug 25 19:28:10 2021 -0700

    HBASE-26138 Backport HBASE-25733 "Upgrade opentelemetry to 1.0.1" to branch-2 (#3628)
    
    15/17 commits of HBASE-22120, original commit b71488998970a3353086a34736ed1edab527f673
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 pom.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/pom.xml b/pom.xml
index 7e08795..e15611e 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1483,8 +1483,8 @@
     <jruby.version>9.2.13.0</jruby.version>
     <junit.version>4.13</junit.version>
     <hamcrest.version>1.3</hamcrest.version>
-    <opentelemetry.version>1.0.0</opentelemetry.version>
-    <opentelemetry-javaagent.version>1.0.0</opentelemetry-javaagent.version>
+    <opentelemetry.version>1.0.1</opentelemetry.version>
+    <opentelemetry-javaagent.version>1.0.1</opentelemetry-javaagent.version>
     <log4j.version>1.2.17</log4j.version>
     <mockito-core.version>2.28.2</mockito-core.version>
     <!--Internally we use a different version of protobuf. See hbase-protocol-shaded-->

[hbase] 12/18: HBASE-26135 Backport HBASE-25616 "Upgrade opentelemetry to 1.0.0" to branch-2 (#3620)

Posted by ta...@apache.org.
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 8f160109cd70466e4c11c8f554c6e258661d3d34
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Tue Aug 24 09:22:37 2021 -0700

    HBASE-26135 Backport HBASE-25616 "Upgrade opentelemetry to 1.0.0" to branch-2 (#3620)
    
    12/17 commits of HBASE-22120, original commits 8399293e21127df3ffdcb757242e4cb5964c7e99
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 pom.xml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/pom.xml b/pom.xml
index 6f04e4d..7e08795 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1483,8 +1483,8 @@
     <jruby.version>9.2.13.0</jruby.version>
     <junit.version>4.13</junit.version>
     <hamcrest.version>1.3</hamcrest.version>
-    <opentelemetry.version>0.17.1</opentelemetry.version>
-    <opentelemetry-javaagent.version>0.17.0</opentelemetry-javaagent.version>
+    <opentelemetry.version>1.0.0</opentelemetry.version>
+    <opentelemetry-javaagent.version>1.0.0</opentelemetry-javaagent.version>
     <log4j.version>1.2.17</log4j.version>
     <mockito-core.version>2.28.2</mockito-core.version>
     <!--Internally we use a different version of protobuf. See hbase-protocol-shaded-->

[hbase] 11/18: HBASE-26134 Backport HBASE-25617 "Revisit the span names" to branch-2 (#3618)

Posted by ta...@apache.org.
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 a2b8f7a3be38050858594c01b5be7171917c6253
Author: Tak Lon (Stephen) Wu <ta...@apache.org>
AuthorDate: Mon Aug 23 21:04:16 2021 -0700

    HBASE-26134 Backport HBASE-25617 "Revisit the span names" to branch-2 (#3618)
    
    11/17 commits of HBASE-22120, original commit 8d68f8cd1c8613be1b499eaa99f46806b2743294
    
    Co-authored-by: Duo Zhang <zh...@apache.org>
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 .../main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java | 4 ++--
 .../main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java  | 2 +-
 .../main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java   | 2 +-
 3 files changed, 4 insertions(+), 4 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 b919ee1..98e7825 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
@@ -386,12 +386,12 @@ class AsyncConnectionImpl implements AsyncConnection {
         }
       });
       return future;
-    }, getClass().getName() + ".getHbck");
+    }, "AsyncConnection.getHbck");
   }
 
   @Override
   public Hbck getHbck(ServerName masterServer) throws IOException {
-    Span span = TraceUtil.createSpan(getClass().getName() + ".getHbck")
+    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
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index 3067785..61086c4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -97,7 +97,7 @@ class AsyncRegionLocator {
 
   private <T> CompletableFuture<T> tracedLocationFuture(Supplier<CompletableFuture<T>> action,
     Function<T, List<String>> getRegionNames, TableName tableName, String methodName) {
-    Span span = createTableSpan(getClass().getSimpleName() + "." + methodName, tableName);
+    Span span = createTableSpan("AsyncRegionLocator." + methodName, tableName);
     try (Scope scope = span.makeCurrent()) {
       CompletableFuture<T> future = action.get();
       FutureUtils.addListener(future, (resp, error) -> {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
index 627e8d2..23bd333 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionFactory.java
@@ -307,6 +307,6 @@ public class ConnectionFactory {
         }
       });
       return future;
-    }, ConnectionFactory.class.getSimpleName() + ".createAsyncConnection");
+    }, "ConnectionFactory.createAsyncConnection");
   }
 }