diff --git a/bin/hbase b/bin/hbase
index 496ebe1ad4b1..5f3b3261f023 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.cmd b/conf/hbase-env.cmd
index 1f1c3e3e6207..4beebf646dee 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 6cc16d8db493..d1d9a6fbbb5a 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
+# 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"
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index 451e9f3e1f16..4ea650d82ce5 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -339,6 +339,12 @@
log4j
log4j
+
+
+ io.opentelemetry.javaagent
+ opentelemetry-javaagent
+ all
+
diff --git a/hbase-assembly/src/main/assembly/client.xml b/hbase-assembly/src/main/assembly/client.xml
index bd65cb43f8e9..48940b75d5be 100644
--- a/hbase-assembly/src/main/assembly/client.xml
+++ b/hbase-assembly/src/main/assembly/client.xml
@@ -65,6 +65,7 @@
org.slf4j:jcl-over-slf4j
org.slf4j:jul-to-slf4j
org.slf4j:slf4j-log4j12
+ io.opentelemetry.javaagent:*
@@ -153,6 +154,13 @@
org.slf4j:jcl-over-slf4j
org.slf4j:jul-to-slf4j
org.slf4j:slf4j-log4j12
+ io.opentelemetry:*
+
+
+
+ lib/trace
+
+ io.opentelemetry.javaagent:*
diff --git a/hbase-assembly/src/main/assembly/hadoop-three-compat.xml b/hbase-assembly/src/main/assembly/hadoop-three-compat.xml
index e5422f39e5f0..6c670a77f7e0 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 @@
org.apache.yetus:audience-annotations
org.slf4j:slf4j-api
org.slf4j:slf4j-log4j12
+ io.opentelemetry.javaagent:*
@@ -212,6 +213,7 @@
org.slf4j:jcl-over-slf4j
org.slf4j:jul-to-slf4j
org.slf4j:slf4j-log4j12
+ io.opentelemetry:*
@@ -256,6 +258,12 @@
jakarta.jws:jakarta.jws-api
+
+ lib/trace
+
+ io.opentelemetry.javaagent:*
+
+
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 9b276aca0785..fc148e8de796 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 @@
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 @@ protected static void startMiniDFSCluster(int servers) throws IOException {
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 08e917427873..01a74cc6188c 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -145,8 +145,8 @@
zookeeper
- org.apache.htrace
- htrace-core4
+ io.opentelemetry
+ opentelemetry-api
org.jruby.jcodings
@@ -156,6 +156,16 @@
org.jruby.joni
joni
+
+ io.opentelemetry
+ opentelemetry-sdk
+ test
+
+
+ io.opentelemetry
+ opentelemetry-sdk-testing
+ test
+
org.slf4j
jcl-over-slf4j
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 7d946aada0b8..54138d30516c 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 @@ private static RegionLocations transformMetaRegionLocations(GetMetaRegionLocatio
@Override
public CompletableFuture getMetaRegionLocations() {
- return this
- . call((c, s, d) -> s.getMetaRegionLocations(c,
- GetMetaRegionLocationsRequest.getDefaultInstance(), d), r -> r.getMetaLocationsCount() != 0,
+ return tracedFuture(
+ () -> this
+ . 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 getClusterId() {
- return this
- . call(
- (c, s, d) -> s.getClusterId(c, GetClusterIdRequest.getDefaultInstance(), d),
- GetClusterIdResponse::hasClusterId, "getClusterId()")
- .thenApply(GetClusterIdResponse::getClusterId);
+ return tracedFuture(
+ () -> this
+ . call(
+ (c, s, d) -> s.getClusterId(c, GetClusterIdRequest.getDefaultInstance(), d),
+ GetClusterIdResponse::hasClusterId, "getClusterId()")
+ .thenApply(GetClusterIdResponse::getClusterId),
+ getClass().getSimpleName() + ".getClusterId");
}
@Override
public CompletableFuture getActiveMaster() {
- return this
- . call(
- (c, s, d) -> s.getActiveMaster(c, GetActiveMasterRequest.getDefaultInstance(), d),
- GetActiveMasterResponse::hasServerName, "getActiveMaster()")
- .thenApply(resp -> ProtobufUtil.toServerName(resp.getServerName()));
+ return tracedFuture(
+ () -> this
+ .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/AsyncConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnection.java
index 75971ad610c7..d04b5f2cebe1 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.
*
- * 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.
*
* 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 2ed73992d7f1..76b1208f5286 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,7 @@
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 java.io.IOException;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
@@ -35,6 +36,7 @@
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;
@@ -48,6 +50,7 @@
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 @@ public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, Stri
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 @@ public Configuration getConfiguration() {
@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 @@ public AsyncTable build() {
@Override
public AsyncTableBuilder getTableBuilder(TableName tableName,
- ExecutorService pool) {
+ ExecutorService pool) {
return new AsyncTableBuilderBase(tableName, connConf) {
@Override
@@ -361,35 +365,44 @@ public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName
@Override
public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName,
- ExecutorService pool) {
+ ExecutorService pool) {
return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool),
RETRY_TIMER);
}
+ private Hbck getHbckInternal(ServerName masterServer) {
+ Span.current().setAttribute(TraceUtil.SERVER_NAME_KEY, masterServer.getServerName());
+ // we will not create a new connection when creating a new protobuf stub, and for hbck there
+ // will be no performance consideration, so for simplification we will create a new stub every
+ // time instead of caching the stub here.
+ return new HBaseHbck(MasterProtos.HbckService.newBlockingStub(
+ rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout)), rpcControllerFactory);
+ }
+
@Override
public CompletableFuture getHbck() {
- CompletableFuture 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 future = new CompletableFuture<>();
+ addListener(registry.getActiveMaster(), (sn, error) -> {
+ if (error != null) {
+ future.completeExceptionally(error);
+ } else {
+ future.complete(getHbckInternal(sn));
}
- }
- });
- return future;
+ });
+ return future;
+ }, "AsyncConnection.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);
+ public Hbck getHbck(ServerName masterServer) {
+ return TraceUtil.trace(new Supplier() {
+
+ @Override
+ public Hbck get() {
+ return getHbckInternal(masterServer);
+ }
+ }, "AsyncConnection.getHbck");
}
@Override
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index d50070ae8a94..61086c4a6690 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 CompletableFuture withTimeout(CompletableFuture future, long timeoutNs,
- Supplier timeoutMsg) {
+ Supplier timeoutMsg) {
if (future.isDone() || timeoutNs <= 0) {
return future;
}
@@ -83,64 +95,100 @@ private boolean isMeta(TableName tableName) {
return TableName.isMetaTableName(tableName);
}
+ private CompletableFuture tracedLocationFuture(Supplier> action,
+ Function> getRegionNames, TableName tableName, String methodName) {
+ Span span = createTableSpan("AsyncRegionLocator." + methodName, tableName);
+ try (Scope scope = span.makeCurrent()) {
+ CompletableFuture future = action.get();
+ FutureUtils.addListener(future, (resp, error) -> {
+ if (error != null) {
+ TraceUtil.setError(span, error);
+ } else {
+ List regionNames = getRegionNames.apply(resp);
+ if (!regionNames.isEmpty()) {
+ span.setAttribute(REGION_NAMES_KEY, regionNames);
+ }
+ span.setStatus(StatusCode.OK);
+ }
+ span.end();
+ });
+ return future;
+ }
+ }
+
+ private List getRegionName(RegionLocations locs) {
+ List names = new ArrayList<>();
+ for (HRegionLocation loc : locs.getRegionLocations()) {
+ if (loc != null) {
+ names.add(loc.getRegion().getRegionNameAsString());
+ }
+ }
+ return names;
+ }
+
CompletableFuture getRegionLocations(TableName tableName, byte[] row,
- RegionLocateType type, boolean reload, long timeoutNs) {
- CompletableFuture 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 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 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 future = new CompletableFuture<>();
- CompletableFuture 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 future = new CompletableFuture<>();
+ CompletableFuture 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 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 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 getRegionLocation(TableName tableName, byte[] row,
- RegionLocateType type, long timeoutNs) {
+ RegionLocateType type, long timeoutNs) {
return getRegionLocation(tableName, row, type, false, timeoutNs);
}
@@ -153,24 +201,31 @@ void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
}
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/AsyncRequestFutureImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRequestFutureImpl.java
index d2486cc1cf0c..8cfcf0c1dbb1 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.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 @@ private Collection extends Runnable> getNewMultiActionRunnable(ServerName serv
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 @@ private Collection extends Runnable> getNewMultiActionRunnable(ServerName serv
List 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 @@ private Collection extends Runnable> getNewMultiActionRunnable(ServerName serv
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/AsyncTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTable.java
index 7473ed0ad919..c7003e052377 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 {
* 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 @@ default CompletableFuture exists(Get get) {
* {@link CompletableFuture}.
*/
default CompletableFuture incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
- long amount) {
+ long amount) {
return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
}
@@ -204,12 +205,12 @@ default CompletableFuture incrementColumnValue(byte[] row, byte[] family,
* {@link CompletableFuture}.
*/
default CompletableFuture 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 @@ interface CheckAndMutateWithFilterBuilder {
}
/**
- * 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 @@ interface CheckAndMutateWithFilterBuilder {
* 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> checkAndMutate(
- List checkAndMutates);
+ List>
+ checkAndMutate(List 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> checkAndMutateAll(
- List checkAndMutates) {
+ default CompletableFuture>
+ checkAndMutateAll(List checkAndMutates) {
return allOf(checkAndMutate(checkAndMutates));
}
@@ -484,7 +481,7 @@ default ResultScanner getScanner(byte[] family, byte[] qualifier) {
*/
default List> exists(List gets) {
return get(toCheckExistenceOnly(gets)).stream()
- .> map(f -> f.thenApply(r -> r.getExists())).collect(toList());
+ .> map(f -> f.thenApply(r -> r.getExists())).collect(toList());
}
/**
@@ -592,7 +589,7 @@ default CompletableFuture> batchAll(List extends Row> actions) {
* @see ServiceCaller
*/
CompletableFuture coprocessorService(Function stubMaker,
- ServiceCaller callable, byte[] row);
+ ServiceCaller callable, byte[] row);
/**
* The callback when we want to execute a coprocessor call on a range of regions.
@@ -731,5 +728,5 @@ default CoprocessorServiceBuilder toRow(byte[] endKey) {
* for more details.
*/
CoprocessorServiceBuilder coprocessorService(Function stubMaker,
- ServiceCaller callable, CoprocessorCallback callback);
+ ServiceCaller callable, CoprocessorCallback 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 fa3ea1ca4dfa..d5b275d2a77e 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 @@ public TableName getName() {
@Override
public CompletableFuture getRegionLocation(byte[] row, int replicaId,
- boolean reload) {
+ boolean reload) {
return conn.getLocator().getRegionLocation(tableName, row, replicaId, RegionLocateType.CURRENT,
reload, -1L);
}
@Override
public CompletableFuture> 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 f91b2107c6b0..23bd333f6612 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.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 static CompletableFuture createAsyncConnection(Configura
*/
public static CompletableFuture createAsyncConnection(Configuration conf,
final User user) {
- CompletableFuture 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 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.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 bed896eaa455..f637e47f60ae 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.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 {
}
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 @@ private static CompletableFuture mutate(HBaseRpcController con
}
private static CompletableFuture voidMutate(HBaseRpcController controller,
- HRegionLocation loc, ClientService.Interface stub, REQ req,
- Converter reqConvert) {
+ HRegionLocation loc, ClientService.Interface stub, REQ req,
+ Converter 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 @@ private interface NoncedConverter {
}
private CompletableFuture noncedMutate(long nonceGroup, long nonce,
- HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub, REQ req,
- NoncedConverter reqConvert,
- Converter respConverter) {
+ HBaseRpcController controller, HRegionLocation loc, ClientService.Interface stub, REQ req,
+ NoncedConverter reqConvert,
+ Converter respConverter) {
return mutate(controller, loc, stub, req,
(info, src) -> reqConvert.convert(info, src, nonceGroup, nonce), respConverter);
}
@@ -240,8 +242,8 @@ private SingleRequestCallerBuilder newCaller(byte[] row, int priority, lo
.maxAttempts(maxAttempts).startLogErrorsCnt(startLogErrorsCnt);
}
- private SingleRequestCallerBuilder newCaller(
- R row, long rpcTimeoutNs) {
+ private SingleRequestCallerBuilder
+ newCaller(R row, long rpcTimeoutNs) {
return newCaller(row.getRow(), row.getPriority(), rpcTimeoutNs);
}
@@ -256,50 +258,58 @@ private CompletableFuture get(Get get, int replicaId) {
@Override
public CompletableFuture 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 put(Put put) {
validatePut(put, conn.connConf.getMaxKeyValueSize());
- return this. newCaller(put, writeRpcTimeoutNs)
+ return tracedFuture(() -> this. newCaller(put, writeRpcTimeoutNs)
.action((controller, loc, stub) -> RawAsyncTableImpl. voidMutate(controller, loc, stub,
put, RequestConverter::buildMutateRequest))
- .call();
+ .call(), "AsyncTable.put", tableName);
}
@Override
public CompletableFuture delete(Delete delete) {
- return this. newCaller(delete, writeRpcTimeoutNs)
- .action((controller, loc, stub) -> RawAsyncTableImpl. voidMutate(controller, loc,
- stub, delete, RequestConverter::buildMutateRequest))
- .call();
+ return tracedFuture(
+ () -> this. newCaller(delete, writeRpcTimeoutNs)
+ .action((controller, loc, stub) -> RawAsyncTableImpl. voidMutate(controller, loc,
+ stub, delete, RequestConverter::buildMutateRequest))
+ .call(),
+ "AsyncTable.delete", tableName);
}
@Override
public CompletableFuture append(Append append) {
checkHasFamilies(append);
- long nonceGroup = conn.getNonceGenerator().getNonceGroup();
- long nonce = conn.getNonceGenerator().newNonce();
- return this. newCaller(append, rpcTimeoutNs)
- .action(
- (controller, loc, stub) -> this. 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. newCaller(append, rpcTimeoutNs)
+ .action((controller, loc, stub) -> this. noncedMutate(nonceGroup, nonce,
+ controller, loc, stub, append, RequestConverter::buildMutateRequest,
+ RawAsyncTableImpl::toResult))
+ .call();
+ }, "AsyncTable.append", tableName);
}
@Override
public CompletableFuture increment(Increment increment) {
checkHasFamilies(increment);
- long nonceGroup = conn.getNonceGenerator().getNonceGroup();
- long nonce = conn.getNonceGenerator().newNonce();
- return this. newCaller(increment, rpcTimeoutNs)
- .action((controller, loc, stub) -> this. 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. newCaller(increment, rpcTimeoutNs)
+ .action((controller, loc, stub) -> this. 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 @@ private void preCheck() {
public CompletableFuture thenPut(Put put) {
validatePut(put, conn.connConf.getMaxKeyValueSize());
preCheck();
- return RawAsyncTableImpl.this. 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. 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 thenDelete(Delete delete) {
preCheck();
- return RawAsyncTableImpl.this. 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. 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 thenMutate(RowMutations mutation) {
preCheck();
validatePutsInRowMutations(mutation, conn.connConf.getMaxKeyValueSize());
- return RawAsyncTableImpl.this. 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
+ . 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 @@ public CheckAndMutateWithFilterBuilder timeRange(TimeRange timeRange) {
@Override
public CompletableFuture thenPut(Put put) {
validatePut(put, conn.connConf.getMaxKeyValueSize());
- return RawAsyncTableImpl.this. newCaller(row, put.getPriority(), rpcTimeoutNs)
+ return tracedFuture(
+ () -> RawAsyncTableImpl.this. 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 thenDelete(Delete delete) {
- return RawAsyncTableImpl.this. 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. 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 thenMutate(RowMutations mutation) {
validatePutsInRowMutations(mutation, conn.connConf.getMaxKeyValueSize());
- return RawAsyncTableImpl.this. 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
+ . 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 @@ public CheckAndMutateWithFilterBuilder checkAndMutate(byte[] row, Filter filter)
@Override
public CompletableFuture 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
+ . 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
+ . newCaller(checkAndMutate.getRow(), rowMutations.getMaxPriority(),
+ rpcTimeoutNs)
+ .action((controller, loc, stub) -> RawAsyncTableImpl.this
+ . 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 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. 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. newCaller(checkAndMutate.getRow(),
- rowMutations.getMaxPriority(), rpcTimeoutNs)
- .action((controller, loc, stub) ->
- RawAsyncTableImpl.this. 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 future = new CompletableFuture<>();
- future.completeExceptionally(new DoNotRetryIOException(
- "CheckAndMutate doesn't support " + checkAndMutate.getAction().getClass().getName()));
- return future;
- }
+ }, "AsyncTable.checkAndMutate", tableName);
}
@Override
- public List> checkAndMutate(
- List checkAndMutates) {
- return batch(checkAndMutates, rpcTimeoutNs).stream()
- .map(f -> f.thenApply(r -> (CheckAndMutateResult) r)).collect(toList());
+ public List>
+ checkAndMutate(List 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 CompletableFuture mutateRow(HBaseRpcController controller,
- HRegionLocation loc, ClientService.Interface stub, RowMutations mutation,
- Converter reqConvert,
- Function respConverter) {
+ HRegionLocation loc, ClientService.Interface stub, RowMutations mutation,
+ Converter reqConvert, Function respConverter) {
CompletableFuture future = new CompletableFuture<>();
try {
byte[] regionName = loc.getRegion().getRegionName();
@@ -537,12 +562,12 @@ public void run(MultiResponse resp) {
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 @@ public CompletableFuture mutateRow(RowMutations mutations) {
validatePutsInRowMutations(mutations, conn.connConf.getMaxKeyValueSize());
long nonceGroup = conn.getNonceGenerator().getNonceGroup();
long nonce = conn.getNonceGenerator().newNonce();
- return this. newCaller(mutations.getRow(), mutations.getMaxPriority(),
- writeRpcTimeoutNs).action((controller, loc, stub) ->
- this. mutateRow(controller, loc, stub, mutations,
- (rn, rm) -> RequestConverter.buildMultiRequest(rn, rm, nonceGroup, nonce),
+ return tracedFuture(
+ () -> this
+ . newCaller(mutations.getRow(), mutations.getMaxPriority(), writeRpcTimeoutNs)
+ .action((controller, loc, stub) -> this. 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 @@ public ResultScanner getScanner(Scan scan) {
@Override
public CompletableFuture> scanAll(Scan scan) {
- CompletableFuture> future = new CompletableFuture<>();
- List scanResults = new ArrayList<>();
- scan(scan, new AdvancedScanResultConsumer() {
+ return tracedFuture(() -> {
+ CompletableFuture> future = new CompletableFuture<>();
+ List 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> get(List gets) {
- return batch(gets, readRpcTimeoutNs);
+ return tracedFutures(() -> batch(gets, readRpcTimeoutNs), "AsyncTable.getList", tableName);
}
@Override
public List> put(List puts) {
- return voidMutate(puts);
+ return tracedFutures(() -> voidMutate(puts), "AsyncTable.putList", tableName);
}
@Override
public List> delete(List deletes) {
- return voidMutate(deletes);
+ return tracedFutures(() -> voidMutate(deletes), "AsyncTable.deleteList", tableName);
}
@Override
public List> batch(List extends Row> actions) {
- return batch(actions, rpcTimeoutNs);
+ return tracedFutures(() -> batch(actions, rpcTimeoutNs), "AsyncTable.batch", tableName);
}
private List> voidMutate(List extends Row> actions) {
@@ -698,7 +727,7 @@ public long getScanTimeout(TimeUnit unit) {
}
private CompletableFuture coprocessorService(Function stubMaker,
- ServiceCaller callable, RegionInfo region, byte[] row) {
+ ServiceCaller callable, RegionInfo region, byte[] row) {
RegionCoprocessorRpcChannelImpl channel = new RegionCoprocessorRpcChannelImpl(conn, tableName,
region, row, rpcTimeoutNs, operationTimeoutNs);
S stub = stubMaker.apply(channel);
@@ -716,7 +745,7 @@ private CompletableFuture coprocessorService(Function s
@Override
public CompletableFuture coprocessorService(Function stubMaker,
- ServiceCaller callable, byte[] row) {
+ ServiceCaller callable, byte[] row) {
return coprocessorService(stubMaker, callable, null, row);
}
@@ -738,9 +767,9 @@ private boolean locateFinished(RegionInfo region, byte[] endKey, boolean endKeyI
}
private void onLocateComplete(Function stubMaker,
- ServiceCaller callable, CoprocessorCallback callback, List locs,
- byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished,
- AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) {
+ ServiceCaller callable, CoprocessorCallback callback, List locs,
+ byte[] endKey, boolean endKeyInclusive, AtomicBoolean locateFinished,
+ AtomicInteger unfinishedRequest, HRegionLocation loc, Throwable error) {
if (error != null) {
callback.onError(error);
return;
@@ -769,7 +798,7 @@ private void onLocateComplete(Function stubMaker,
}
private final class CoprocessorServiceBuilderImpl
- implements CoprocessorServiceBuilder {
+ implements CoprocessorServiceBuilder {
private final Function stubMaker;
@@ -786,7 +815,7 @@ private final class CoprocessorServiceBuilderImpl
private boolean endKeyInclusive;
public CoprocessorServiceBuilderImpl(Function stubMaker,
- ServiceCaller callable, CoprocessorCallback callback) {
+ ServiceCaller callable, CoprocessorCallback 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 @@ public void execute() {
@Override
public CoprocessorServiceBuilder coprocessorService(
- Function stubMaker, ServiceCaller callable,
- CoprocessorCallback callback) {
+ Function stubMaker, ServiceCaller callable,
+ CoprocessorCallback callback) {
return new CoprocessorServiceBuilderImpl<>(stubMaker, callable, callback);
}
}
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 965b13c2134f..4a96954b21a0 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.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 ResultBoundedCompletionService(
public void submit(RetryingCallable task, int callTimeout, int id) {
QueueingFuture 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/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
index 35107cab2ba7..6e94afe00a79 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.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 @@ private static String getClusterId(byte[] data) throws DeserializationException
@Override
public CompletableFuture getClusterId() {
- return getAndConvert(znodePaths.clusterIdZNode, ZKConnectionRegistry::getClusterId);
+ return tracedFuture(
+ () -> getAndConvert(znodePaths.clusterIdZNode, ZKConnectionRegistry::getClusterId),
+ "ZKConnectionRegistry.getClusterId");
}
ReadOnlyZKClient getZKClient() {
@@ -204,19 +207,20 @@ private void getMetaRegionLocation(CompletableFuture future,
@Override
public CompletableFuture getMetaRegionLocations() {
- CompletableFuture future = new CompletableFuture<>();
- addListener(
- zk.list(znodePaths.baseZNode)
- .thenApply(children -> children.stream()
+ return tracedFuture(() -> {
+ CompletableFuture 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 @@ private static ZooKeeperProtos.Master getMasterProto(byte[] data) throws IOExcep
@Override
public CompletableFuture 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 @@ public CompletableFuture getActiveMaster() {
HBaseProtos.ServerName snProto = proto.getMaster();
return ServerName.valueOf(snProto.getHostName(), snProto.getPort(),
snProto.getStartCode());
- });
+ }),
+ "ZKConnectionRegistry.getActiveMaster");
}
@Override
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 4bbb729f1f2d..bb7f4af57376 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 @@
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.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 @@ private T getConnection(ConnectionId remoteId) throws IOException {
protected abstract T createConnection(ConnectionId remoteId) throws IOException;
private void onCallFinished(Call call, HBaseRpcController hrc, Address addr,
- RpcCallback callback) {
+ RpcCallback callback) {
call.callStats.setCallTimeMs(EnvironmentEdgeManager.currentTime() - call.getStartTime());
if (metrics != null) {
metrics.updateRpc(call.md, call.param, call.callStats);
@@ -388,44 +392,61 @@ private void onCallFinished(Call call, HBaseRpcController hrc, Address addr,
}
}
- Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
- final Message param, Message returnType, final User ticket,
- final Address addr, final RpcCallback 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();
- }
+ private Call callMethod(final Descriptors.MethodDescriptor md, final HBaseRpcController hrc,
+ final Message param, Message returnType, final User ticket, final Address addr,
+ final RpcCallback callback) {
+ 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())
+ .setAttribute(TraceUtil.REMOTE_PORT_KEY, addr.getPort());
+ 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() {
@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()) {
+ TraceUtil.setError(span, 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 cd8035fd58ec..eb8e1d92b216 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,7 @@
import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
import static org.apache.hadoop.hbase.ipc.IPCUtil.write;
+import io.opentelemetry.context.Scope;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
import java.io.DataInputStream;
@@ -54,7 +55,6 @@
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;
@@ -62,7 +62,6 @@
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;
@@ -190,8 +189,8 @@ public void run() {
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.
@@ -592,13 +591,6 @@ private void negotiateCryptoAes(RPCProtos.CryptoCipherMeta cryptoCipherMeta)
this.out = new DataOutputStream(new BufferedOutputStream(saslRpcClient.getOutputStream()));
}
- private void tracedWriteRequest(Call call) throws IOException {
- try (TraceScope ignored = TraceUtil.createTrace("RpcClientImpl.tracedWriteRequest",
- call.span)) {
- writeRequest(call);
- }
- }
-
/**
* Initiates a call by sending the parameter to the remote server. Note: this is not called from
* the Connection thread, but by other threads.
@@ -806,7 +798,9 @@ public void run(boolean cancelled) throws IOException {
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 7793680ca540..8d23d9243399 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.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. */
@@ -60,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 callback, MetricsConnection.CallStats callStats) {
this.param = param;
@@ -73,7 +74,7 @@ protected Call(int id, final Descriptors.MethodDescriptor md, Message param,
this.timeout = timeout;
this.priority = priority;
this.callback = callback;
- this.span = Tracer.getCurrentSpan();
+ this.span = Span.current();
}
/**
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 42ad33a1e614..fd42214d1d30 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.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 @@ public static int getTotalSizeWhenWrittenDelimited(Message... messages) {
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 f31e3d21aabe..c67d96f0a756 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 @@ private void writeRequest(ChannelHandlerContext ctx, Call call, ChannelPromise p
@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-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 000000000000..fec5f6d1e916
--- /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 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-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 000000000000..708ae4caa237
--- /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 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 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 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 000000000000..a3ad9df11544
--- /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.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.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() {
+
+ @Override
+ public Void answer(InvocationOnMock invocation) throws Throwable {
+ ScanRequest req = invocation.getArgument(1);
+ RpcCallback 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() {
+
+ @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 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() {
+
+ @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 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() {
+
+ @Override
+ public Void answer(InvocationOnMock invocation) throws Throwable {
+ RpcCallback 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> answer =
+ new Answer>() {
+
+ @Override
+ public CompletableFuture 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() == SpanKind.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() == 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 64007c0bf5be..2212fd14a416 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -192,8 +192,12 @@
- org.apache.htrace
- htrace-core4
+ io.opentelemetry
+ opentelemetry-api
+
+
+ io.opentelemetry
+ opentelemetry-semconv
org.apache.commons
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 03d03d9fe4e6..000000000000
--- 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 b967db7f27dc..000000000000
--- 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 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 10665d898265..fb37080a5fb9 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,213 @@
*/
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.GlobalOpenTelemetry;
+import io.opentelemetry.api.common.AttributeKey;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.SpanKind;
+import io.opentelemetry.api.trace.StatusCode;
+import io.opentelemetry.api.trace.Tracer;
+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;
+import java.util.function.Supplier;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.util.FutureUtils;
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 static final String INSTRUMENTATION_NAME = "io.opentelemetry.contrib.hbase";
+
+ public static final AttributeKey NAMESPACE_KEY = SemanticAttributes.DB_HBASE_NAMESPACE;
+
+ public static final AttributeKey TABLE_KEY = AttributeKey.stringKey("db.hbase.table");
+
+ public static final AttributeKey> REGION_NAMES_KEY =
+ AttributeKey.stringArrayKey("db.hbase.regions");
+
+ public static final AttributeKey RPC_SERVICE_KEY =
+ AttributeKey.stringKey("db.hbase.rpc.service");
+
+ public static final AttributeKey RPC_METHOD_KEY =
+ AttributeKey.stringKey("db.hbase.rpc.method");
+
+ public static final AttributeKey SERVER_NAME_KEY =
+ AttributeKey.stringKey("db.hbase.server.name");
+
+ public static final AttributeKey REMOTE_HOST_KEY = SemanticAttributes.NET_PEER_NAME;
+
+ public static final AttributeKey REMOTE_PORT_KEY = SemanticAttributes.NET_PEER_PORT;
+
+ public static final AttributeKey ROW_LOCK_READ_LOCK_KEY =
+ AttributeKey.booleanKey("db.hbase.rowlock.readlock");
+
+ public static final AttributeKey WAL_IMPL = AttributeKey.stringKey("db.hbase.wal.impl");
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();
- }
+ public static Tracer getGlobalTracer() {
+ return GlobalOpenTelemetry.getTracer(INSTRUMENTATION_NAME);
}
/**
- * Wrapper method to create new TraceScope with the given description
- * @return TraceScope or null when not tracing
+ * Create a {@link SpanKind#INTERNAL} span.
*/
- public static TraceScope createTrace(String description) {
- return (tracer == null) ? null : tracer.newScope(description);
+ public static Span createSpan(String name) {
+ return createSpan(name, SpanKind.INTERNAL);
}
/**
- * 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
+ * Create a {@link SpanKind#INTERNAL} span and set table related attributes.
*/
- public static TraceScope createTrace(String description, Span span) {
- if (span == null) {
- return createTrace(description);
- }
-
- return (tracer == null) ? null : tracer.newScope(description, span.getSpanId());
+ public static Span createTableSpan(String spanName, TableName tableName) {
+ return createSpan(spanName).setAttribute(NAMESPACE_KEY, tableName.getNamespaceAsString())
+ .setAttribute(TABLE_KEY, tableName.getNameAsString());
}
/**
- * Wrapper method to add new sampler to the default tracer
- * @return true if added, false if it was already added
+ * Create a span with the given {@code kind}. Notice that, OpenTelemetry only expects one
+ * {@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}.
*/
- public static boolean addSampler(Sampler sampler) {
- if (sampler == null) {
- return false;
- }
+ private static Span createSpan(String name, SpanKind kind) {
+ return getGlobalTracer().spanBuilder(name).setSpanKind(kind).startSpan();
+ }
- return (tracer == null) ? false : tracer.addSampler(sampler);
+ /**
+ * Create a span which parent is from remote, i.e, passed through rpc.
+ *
+ * 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(SpanKind.SERVER)
+ .startSpan();
}
/**
- * Wrapper method to add key-value pair to TraceInfo of actual span
+ * Create a span with {@link SpanKind#CLIENT}.
*/
- public static void addKVAnnotation(String key, String value){
- Span span = Tracer.getCurrentSpan();
- if (span != null) {
- span.addKVAnnotation(key, value);
- }
+ public static Span createClientSpan(String name) {
+ return createSpan(name, SpanKind.CLIENT);
}
/**
- * Wrapper method to add receiver to actual tracerpool
- * @return true if successfull, false if it was already added
+ * Trace an asynchronous operation for a table.
*/
- public static boolean addReceiver(SpanReceiver rcvr) {
- return (tracer == null) ? false : tracer.getTracerPool().addReceiver(rcvr);
+ public static CompletableFuture tracedFuture(Supplier> action,
+ String spanName, TableName tableName) {
+ Span span = createTableSpan(spanName, tableName);
+ try (Scope scope = span.makeCurrent()) {
+ CompletableFuture future = action.get();
+ endSpan(future, span);
+ return future;
+ }
}
/**
- * Wrapper method to remove receiver from actual tracerpool
- * @return true if removed, false if doesn't exist
+ * Trace an asynchronous operation.
*/
- public static boolean removeReceiver(SpanReceiver rcvr) {
- return (tracer == null) ? false : tracer.getTracerPool().removeReceiver(rcvr);
+ public static CompletableFuture tracedFuture(Supplier> action,
+ String spanName) {
+ Span span = createSpan(spanName);
+ try (Scope scope = span.makeCurrent()) {
+ CompletableFuture future = action.get();
+ endSpan(future, span);
+ return future;
+ }
}
/**
- * Wrapper method to add timeline annotiation to current span with given message
+ * Trace an asynchronous operation, and finish the create {@link Span} when all the given
+ * {@code futures} are completed.
*/
- public static void addTimelineAnnotation(String msg) {
- Span span = Tracer.getCurrentSpan();
- if (span != null) {
- span.addTimelineAnnotation(msg);
+ public static List> tracedFutures(
+ Supplier>> action, String spanName, TableName tableName) {
+ Span span = createTableSpan(spanName, tableName);
+ try (Scope scope = span.makeCurrent()) {
+ List> futures = action.get();
+ endSpan(CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])), span);
+ return futures;
}
}
+ public static void setError(Span span, Throwable error) {
+ span.recordException(error);
+ span.setStatus(StatusCode.ERROR);
+ }
+
/**
- * Wrap runnable with current tracer and description
- * @param runnable to wrap
- * @return wrapped runnable or original runnable when not tracing
+ * Finish the {@code span} when the given {@code future} is completed.
*/
- public static Runnable wrap(Runnable runnable, String description) {
- return (tracer == null) ? runnable : tracer.wrap(runnable, description);
+ private static void endSpan(CompletableFuture> future, Span span) {
+ FutureUtils.addListener(future, (resp, error) -> {
+ if (error != null) {
+ setError(span, 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 creator) {
+ Span span = creator.get();
+ try (Scope scope = span.makeCurrent()) {
+ action.run();
+ span.setStatus(StatusCode.OK);
+ } catch (Throwable e) {
+ setError(span, e);
+ throw e;
+ } finally {
+ span.end();
+ }
+ }
+
+ public static T trace(Supplier 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 call() throws IOException;
+ }
+
+ public static T trace(IOExceptionCallable callable, String spanName) throws IOException {
+ return trace(callable, () -> createSpan(spanName));
+ }
+
+ public static T trace(IOExceptionCallable callable, Supplier 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-external-blockcache/pom.xml b/hbase-external-blockcache/pom.xml
index 0b37d5dc035b..1cdffc873b14 100644
--- a/hbase-external-blockcache/pom.xml
+++ b/hbase-external-blockcache/pom.xml
@@ -109,10 +109,6 @@
org.slf4j
slf4j-api
-
- org.apache.htrace
- htrace-core4
-
junit
junit
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 246d7e0a138c..350d2c72f798 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.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.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 void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
}
@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 Cacheable getBlock(BlockCacheKey cacheKey, boolean caching,
}
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 d1213a009d2e..d4ef4ec67c80 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -247,8 +247,8 @@
commons-lang3
- org.apache.htrace
- htrace-core4
+ io.opentelemetry
+ opentelemetry-api
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 e21dfecab98f..1cd43047b570 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 void testAddDependencyJars() throws Exception {
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 d946045d628e..13e0f0ae888c 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 @@
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;
@@ -52,7 +54,6 @@
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;
@@ -64,9 +65,6 @@
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 +374,9 @@ private static class TimingResult {
* @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().getTraceId());
}
}
@@ -421,15 +416,11 @@ public TimingResult call() throws Exception {
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 +451,6 @@ public TimingResult call() throws Exception {
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 +465,8 @@ public TimingResult call() throws Exception {
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 ea219db9b028..e9f3aa062e35 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.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 int doWork() throws Exception {
public void internalDoWork() throws Exception {
util = createUtil();
admin = util.getAdmin();
- setupReceiver();
deleteTable();
createTable();
@@ -108,51 +110,53 @@ public void internalDoWork() throws Exception {
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 rks) {
+ for (int i = 0; i < 100; i++) {
+ Runnable runnable = new Runnable() {
+ private final LinkedBlockingQueue 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 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 rowKeys)
@@ -173,9 +177,9 @@ public void run() {
}
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 void run() {
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 void run() {
}
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 @@ private LinkedBlockingQueue insertData() throws IOException, InterruptedEx
LinkedBlockingQueue 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 @@ private LinkedBlockingQueue insertData() throws IOException, InterruptedEx
if ((x % 1000) == 0) {
admin.flush(tableName);
}
+ } finally {
+ span.end();
}
}
admin.flush(tableName);
@@ -255,11 +267,4 @@ private IntegrationTestingUtility createUtil() throws Exception {
}
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 e5019f47d71a..4f359a849915 100644
--- a/hbase-mapreduce/pom.xml
+++ b/hbase-mapreduce/pom.xml
@@ -155,8 +155,8 @@
slf4j-api
- org.apache.htrace
- htrace-core4
+ io.opentelemetry
+ opentelemetry-api
org.apache.hbase
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 fe8031cf31b6..d8bffb31ec5e 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,9 +831,10 @@ public static void addHBaseDependencyJars(Configuration conf) throws IOException
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
+ 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
}
/**
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 9cbae3377a19..68967d63f2d4 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 @@
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.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.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 @@ static class TestOptions {
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 @@ private static long nextRandomSeed() {
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 @@ private static long nextRandomSeed() {
*/
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 @@ void testTakedown() throws IOException {
YammerHistogramUtils.getHistogramReport(bytesInRemoteResultsHistogram));
}
}
- receiverHost.closeReceivers();
}
abstract void onTakedown() throws IOException;
@@ -1388,7 +1377,6 @@ int getLastRow() {
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 @@ void testTimed() throws IOException, InterruptedException {
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 9a0b55311d94..b9fd6063f80a 100644
--- a/hbase-protocol-shaded/pom.xml
+++ b/hbase-protocol-shaded/pom.xml
@@ -201,10 +201,6 @@
junit
test
-
- org.apache.htrace
- htrace-core4
-
diff --git a/hbase-protocol-shaded/src/main/protobuf/Tracing.proto b/hbase-protocol-shaded/src/main/protobuf/Tracing.proto
index 64ead844d950..276a0a77b958 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 headers = 3;
}
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index b0ee3e78f340..ef2fad381740 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -427,8 +427,8 @@
- org.apache.htrace
- htrace-core4
+ io.opentelemetry
+ opentelemetry-api
com.lmax
@@ -440,6 +440,16 @@
hamcrest-core
test
+
+ io.opentelemetry
+ opentelemetry-sdk
+ test
+
+
+ io.opentelemetry
+ opentelemetry-sdk-testing
+ test
+
org.hamcrest
hamcrest-library
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 df84e004503f..17054a5c409c 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
* 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 EventHandler prepare() throws Exception {
@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 7375fa8ddb5b..0bb8d23348f0 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.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 HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize,
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 HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize,
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 HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize,
// 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 HFileBlock readBlock(long dataBlockOffset, long onDiskBlockSize,
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 e5354d7db8d0..6e2c37e3a8c5 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,26 @@
*/
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;
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;
/**
@@ -72,15 +75,6 @@ public RpcCall getRpcCall() {
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;
}
@@ -94,6 +88,14 @@ private void cleanup() {
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,24 +120,26 @@ public void run() {
String error = null;
Pair resultPair = null;
RpcServer.CurCall.set(call);
- TraceScope traceScope = null;
- try {
+ String serviceName = getServiceName();
+ String methodName = getMethodName();
+ 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();
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){
RpcServer.LOG.warn("Can not complete this request in time, drop it: " + call);
+ TraceUtil.setError(span, e);
return;
} catch (Throwable e) {
+ TraceUtil.setError(span, e);
if (e instanceof ServerNotRunningYetException) {
// If ServerNotRunningYetException, don't spew stack trace.
if (RpcServer.LOG.isTraceEnabled()) {
@@ -151,14 +155,13 @@ public void run() {
throw (Error)e;
}
} finally {
- if (traceScope != null) {
- traceScope.close();
- }
RpcServer.CurCall.set(null);
if (resultPair != null) {
this.rpcServer.addCallSize(call.getSize() * -1);
+ span.setStatus(StatusCode.OK);
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/ipc/ServerCall.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerCall.java
index ff4a521d3b06..53101c9e9f4b 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;
@@ -31,6 +33,7 @@
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;
@@ -102,6 +105,8 @@ public abstract class ServerCall 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 +137,7 @@ public abstract class ServerCall implements RpcCa
this.bbAllocator = byteBuffAllocator;
this.cellBlockBuilder = cellBlockBuilder;
this.reqCleanup = reqCleanup;
+ this.span = Span.current();
}
/**
@@ -150,6 +156,7 @@ public void done() {
// 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 +233,9 @@ public synchronized void setResponse(Message m, final CellScanner cells, Throwab
}
if (t != null) {
this.isError = true;
+ TraceUtil.setError(span, t);
+ } else {
+ span.setStatus(StatusCode.OK);
}
BufferChain bc = null;
try {
@@ -560,4 +570,8 @@ public synchronized BufferChain getResponse() {
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 29ce30b86529..208ec8ba8f17 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 @@
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.TextMapGetter;
import java.io.ByteArrayInputStream;
import java.io.Closeable;
import java.io.DataOutputStream;
@@ -31,13 +36,11 @@
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.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.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.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,114 @@ protected void processRequest(ByteBuff buf) throws IOException,
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);
+ TextMapGetter getter = new TextMapGetter() {
+
+ @Override
+ public Iterable 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.createRemoteSpan("RpcServer.process", traceCtx);
+ 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/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index 4d8721add2b3..24277814dd41 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.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 6c3ee1db652c..0f0a2b655b1b 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 @@
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.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 int run(String args[]) throws Exception {
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 20c52c3b978e..6ab6f14a4a9a 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,7 @@
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 java.io.EOFException;
import java.io.FileNotFoundException;
import java.io.IOException;
@@ -191,7 +192,6 @@
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;
@@ -3110,24 +3110,26 @@ public RegionScannerImpl getScanner(Scan scan, List additionalS
}
private RegionScannerImpl getScanner(Scan scan, List 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,
@@ -3164,15 +3166,17 @@ private void prepareDelete(Delete delete) throws IOException {
@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"));
}
/**
@@ -3247,20 +3251,22 @@ private void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, b
@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"));
}
/**
@@ -3541,7 +3547,7 @@ public MiniBatchOperationInProgress lockRowsAndBuildMiniBatch(
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
@@ -4325,7 +4331,7 @@ private void checkAndMergeCPMutations(final MiniBatchOperationInProgress batchMutate(mutations, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE),
+ () -> createRegionSpan("Region.batchMutate"));
}
public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId)
@@ -4778,6 +4786,12 @@ public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws
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;
@@ -4826,7 +4840,7 @@ public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate, long n
}
// 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 =
@@ -4836,7 +4850,7 @@ public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate, long n
}
}
- // 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;
@@ -4884,7 +4898,7 @@ public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate, long n
// 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);
}
@@ -4912,8 +4926,7 @@ public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate, long n
}
}
- 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(
@@ -6561,11 +6574,17 @@ public RowLock getRowLock(byte[] row) throws IOException {
@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);
@@ -6573,8 +6592,7 @@ protected RowLock getRowLockInternal(byte[] row, boolean readLock, final RowLock
RowLockImpl result = null;
boolean success = false;
- try (TraceScope scope = TraceUtil.createTrace("HRegion.getRowLock")) {
- TraceUtil.addTimelineAnnotation("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) {
@@ -6611,7 +6629,6 @@ protected RowLock getRowLockInternal(byte[] row, boolean readLock, final RowLock
}
if (timeout <= 0 || !result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS)) {
- TraceUtil.addTimelineAnnotation("Failed to get row lock");
String message = "Timed out waiting for lock for row: " + rowKey + " in region "
+ getRegionInfo().getEncodedName();
if (reachDeadlineFirst) {
@@ -6629,7 +6646,6 @@ protected RowLock getRowLockInternal(byte[] row, boolean readLock, final RowLock
LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey,
getRegionInfo().getRegionNameAsString());
}
- TraceUtil.addTimelineAnnotation("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,7 +6654,6 @@ protected RowLock getRowLockInternal(byte[] row, boolean readLock, final RowLock
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");
throw ioe;
} finally {
// Clean up the counts just in case this was the thing keeping the context alive.
@@ -6648,6 +6663,13 @@ protected RowLock getRowLockInternal(byte[] row, boolean readLock, final RowLock
}
}
+ 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 rowLocks) {
if (rowLocks != null) {
for (RowLock rowLock : rowLocks) {
@@ -7511,9 +7533,15 @@ public List get(Get get, boolean withCoprocessor) throws IOException {
}
private List 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 getInternal(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
+ throws IOException {
List| results = new ArrayList<>();
- long before = EnvironmentEdgeManager.currentTime();
+ long before = EnvironmentEdgeManager.currentTime();
// pre-get CP hook
if (withCoprocessor && (coprocessorHost != null)) {
@@ -7610,7 +7638,7 @@ public MiniBatchOperationInProgress lockRowsAndBuildMiniBatch(
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;
@@ -7878,15 +7906,17 @@ public Result append(Append append) throws IOException {
}
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
@@ -7895,15 +7925,17 @@ public Result increment(Increment increment) throws IOException {
}
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 clusterIds,
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 3e4a501baec0..a3004878e870 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.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;
@@ -399,7 +397,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
@@ -596,7 +593,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;
@@ -668,7 +664,6 @@ public HRegionServer(final Configuration conf) throws IOException {
(t, e) -> abort("Uncaught exception in executorService thread " + t.getName(), e);
initializeFileSystem();
- spanReceiverHost = SpanReceiverHost.getInstance(getConfiguration());
this.configurationManager = new ConfigurationManager();
setupWindows(getConfiguration(), getConfigurationManager());
@@ -2717,10 +2712,6 @@ protected void stopServiceThreads() {
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 afd85f8d7895..5fd12333ad91 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 @@ protected String getUsage() {
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 77755bd3060b..08563570dbed 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.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.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 @@ private int getStoreFileCount(Region region) {
* 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 void reclaimMemStoreMemory() {
if (flushType != FlushType.NORMAL) {
wakeupFlushThread();
}
+ span.end();
}
}
}
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 612bb5c9e371..881f4234bf9c 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.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;
@@ -723,8 +724,9 @@ protected boolean shouldStop(Cell currentRowCell) {
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;
@@ -738,24 +740,31 @@ public synchronized void close() {
this.filterClosed = true;
}
+ @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/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 0d9c14048bf6..c39e10dc5540 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,7 @@
import static org.apache.hbase.thirdparty.com.google.common.base.Preconditions.checkNotNull;
import com.lmax.disruptor.RingBuffer;
+import io.opentelemetry.api.trace.Span;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InterruptedIOException;
@@ -83,7 +84,6 @@
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;
@@ -548,6 +548,35 @@ public Map> rollWriter() throws FailedLogCloseException, IO
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
@@ -649,7 +678,7 @@ Map> findRegionsToForceFlush() throws IOException {
this.sequenceIdAccounting.findLower(firstWALEntry.getValue().encodedName2HighestSequenceId);
}
if (regions != null) {
- List listForPrint = new ArrayList();
+ List listForPrint = new ArrayList<>();
for (Map.Entry> r : regions.entrySet()) {
StringBuilder families = new StringBuilder();
for (int i = 0; i < r.getValue().size(); i++) {
@@ -767,6 +796,10 @@ protected final void logRollAndSetupWalProps(Path oldPath, Path newPath, long ol
}
}
+ 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}.
*
@@ -784,10 +817,10 @@ protected final void logRollAndSetupWalProps(Path oldPath, Path newPath, long ol
* @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")) {
+ return TraceUtil.trace(() -> {
doReplaceWriter(oldPath, newPath, nextWriter);
return newPath;
- }
+ }, () -> createSpan("WAL.replaceWriter"));
}
protected final void blockOnSync(SyncFuture syncFuture) throws IOException {
@@ -821,8 +854,7 @@ private IOException convertInterruptedExceptionToIOException(final InterruptedEx
return ioe;
}
- @Override
- public Map> rollWriter(boolean force) throws IOException {
+ private Map> rollWriterInternal(boolean force) throws IOException {
rollWriterLock.lock();
try {
// Return if nothing to flush.
@@ -834,7 +866,7 @@ public Map> rollWriter(boolean force) throws IOException {
LOG.debug("WAL closed. Skipping rolling of writer");
return regionsToFlush;
}
- try (TraceScope scope = TraceUtil.createTrace("FSHLog.rollWriter")) {
+ try {
Path oldPath = getOldPath();
Path newPath = getNewPath();
// Any exception from here on is catastrophic, non-recoverable so we currently abort.
@@ -859,8 +891,8 @@ public Map> rollWriter(boolean force) throws IOException {
// 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);
+ "Underlying FileSystem can't meet stream requirements. See RS log " + "for details.",
+ exception);
}
return regionsToFlush;
} finally {
@@ -868,6 +900,11 @@ public Map> rollWriter(boolean force) throws IOException {
}
}
+ @Override
+ public Map> 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() {
@@ -1052,7 +1089,6 @@ protected final void postSync(final long timeInNanos, final int handlerSyncs) {
.append(TimeUnit.NANOSECONDS.toMillis(timeInNanos))
.append(" ms, current pipeline: ")
.append(Arrays.toString(getPipeline())).toString();
- TraceUtil.addTimelineAnnotation(msg);
LOG.info(msg);
// A single sync took too long.
// Elsewhere in checkSlowSync, called from checkLogRoll, we will look at cumulative
@@ -1075,8 +1111,7 @@ protected final void postSync(final long timeInNanos, final int handlerSyncs) {
}
protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKeyImpl key,
- WALEdit edits, boolean inMemstore, RingBuffer ringBuffer)
- throws IOException {
+ WALEdit edits, boolean inMemstore, RingBuffer ringBuffer) throws IOException {
if (this.closed) {
throw new IOException(
"Cannot append; log is closed, regionName = " + hri.getRegionNameAsString());
@@ -1088,7 +1123,7 @@ protected final long stampSequenceIdAndPublishToRingBuffer(RegionInfo hri, WALKe
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")) {
+ try {
FSWALEntry entry = new FSWALEntry(txid, key, edits, hri, inMemstore, rpcCall);
entry.stampRegionSequenceId(we);
ringBuffer.get(txid).load(entry);
@@ -1136,13 +1171,14 @@ public OptionalLong getLogFileSizeIfBeingWritten(Path path) {
@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 ae26a47a494c..54532b5395d7 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,7 +20,6 @@
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;
@@ -44,7 +43,6 @@
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;
@@ -52,18 +50,16 @@
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;
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;
@@ -350,7 +346,7 @@ private void syncCompleted(AsyncWriter writer, long processedTxid, long startTim
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.
@@ -400,13 +396,7 @@ private void sync(AsyncWriter writer) {
}, consumeExecutor);
}
- private void addTimeAnnotation(SyncFuture future, String annotation) {
- TraceUtil.addTimelineAnnotation(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 iter = syncFutures.iterator(); iter.hasNext();) {
SyncFuture sync = iter.next();
@@ -414,9 +404,6 @@ private int finishSyncLowerThanTxid(long txid, boolean addSyncTrace) {
markFutureDoneAndOffer(sync, txid, null);
iter.remove();
finished++;
- if (addSyncTrace) {
- addTimeAnnotation(sync, "writer synced");
- }
} else {
break;
}
@@ -425,7 +412,7 @@ private int finishSyncLowerThanTxid(long txid, boolean addSyncTrace) {
}
// 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()) {
@@ -433,10 +420,7 @@ private int finishSync(boolean addSyncTrace) {
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();
@@ -450,7 +434,7 @@ private int finishSync(boolean addSyncTrace) {
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
@@ -458,7 +442,7 @@ private int finishSync(boolean addSyncTrace) {
long lowestUnackedAppendTxid = unackedAppends.peek().getTxid();
long doneTxid = Math.max(lowestUnackedAppendTxid - 1, highestSyncedTxid.get());
highestSyncedTxid.set(doneTxid);
- return finishSyncLowerThanTxid(doneTxid, addSyncTrace);
+ return finishSyncLowerThanTxid(doneTxid);
}
}
@@ -466,7 +450,7 @@ private void appendAndSync() {
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 iter = toWriteAppends.iterator(); iter.hasNext();) {
FSWALEntry entry = iter.next();
@@ -507,7 +491,7 @@ private void appendAndSync() {
// stamped some region sequence id.
if (unackedAppends.isEmpty()) {
highestSyncedTxid.set(highestProcessedAppendTxid);
- finishSync(false);
+ finishSync();
trySetReadyForRolling();
}
return;
@@ -613,55 +597,41 @@ protected long append(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inM
}
@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 {
- try (TraceScope scope = TraceUtil.createTrace("AsyncFSWAL.sync")) {
- 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 {
+ 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;
}
- try (TraceScope scope = TraceUtil.createTrace("AsyncFSWAL.sync")) {
- // 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 {
+ 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 690f54520ae9..8d61cf940f87 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,7 @@
import com.lmax.disruptor.TimeoutException;
import com.lmax.disruptor.dsl.Disruptor;
import com.lmax.disruptor.dsl.ProducerType;
+import io.opentelemetry.api.trace.Span;
import java.io.IOException;
import java.io.OutputStream;
import java.util.Arrays;
@@ -47,7 +48,6 @@
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;
@@ -59,7 +59,6 @@
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 +363,6 @@ protected void doReplaceWriter(Path oldPath, Path newPath, Writer nextWriter) th
// 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");
syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer(sequence, false));
}
} catch (FailedSyncBeforeLogCloseException e) {
@@ -436,10 +434,11 @@ protected void doReplaceWriter(Path oldPath, Path newPath, Writer nextWriter) th
}
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();
@@ -644,15 +643,11 @@ public void run() {
}
// 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 {
- TraceUtil.addTimelineAnnotation("syncing writer");
long unSyncedFlushSeq = highestUnsyncedTxid;
writer.sync(sf.isForceSync());
- TraceUtil.addTimelineAnnotation("writer synced");
if (unSyncedFlushSeq > currentSequence) {
currentSequence = unSyncedFlushSeq;
}
@@ -664,9 +659,6 @@ public void run() {
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?
@@ -791,7 +783,7 @@ protected SyncFuture publishSyncOnRingBuffer(long sequence, boolean forceSync) {
}
// Sync all known transactions
- private void publishSyncThenBlockOnCompletion(TraceScope scope, boolean forceSync) throws IOException {
+ private void publishSyncThenBlockOnCompletion(boolean forceSync) throws IOException {
SyncFuture syncFuture = publishSyncOnRingBuffer(forceSync);
blockOnSync(syncFuture);
}
@@ -817,31 +809,17 @@ int getLogReplication() {
}
@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 {
- try (TraceScope scope = TraceUtil.createTrace("FSHLog.sync")) {
- 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;
}
- try (TraceScope scope = TraceUtil.createTrace("FSHLog.sync")) {
- publishSyncThenBlockOnCompletion(scope, forceSync);
- }
+ publishSyncThenBlockOnCompletion(forceSync);
}
boolean isLowReplicationRollEnabled() {
@@ -1062,8 +1040,6 @@ public void onEvent(final RingBufferTruck truck, final long sequence, boolean en
}
} 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/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java
index 1c4636eb00e1..f67fece8aee5 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 @@
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.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.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.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 MiniDFSCluster startMiniDFSCluster(int servers, final String racks[], St
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 MiniHBaseCluster startMiniHBaseCluster(StartMiniClusterOption option)
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 6b58d073e031..5df089b597ec 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 static class TestEventHandler extends EventHandler {
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/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index 87561bac745b..ce753430b333 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,29 @@
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.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.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 +51,13 @@
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.trace.TraceUtil;
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 +98,10 @@ protected abstract RpcServer createRpcServer(final Server server, final String n
protected abstract AbstractRpcClient> createRpcClientNoCodec(Configuration conf);
+
+ @Rule
+ public OpenTelemetryRule traceRule = OpenTelemetryRule.create();
+
/**
* Ensure we do not HAVE TO HAVE a codec.
*/
@@ -183,7 +198,7 @@ public void testRpcScheduler() throws IOException, ServiceException, Interrupted
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 +207,7 @@ public void testRpcScheduler() throws IOException, ServiceException, Interrupted
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 +442,70 @@ public void testAsyncTimeout() throws IOException {
}
}
+ 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());
+ }
+ }
+
+ 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, InetSocketAddress addr,
+ SpanKind kind) {
+ assertEquals(SERVICE.getDescriptorForType().getName(),
+ data.getAttributes().get(TraceUtil.RPC_SERVICE_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());
+ }
+ assertEquals(kind, data.getKind());
+ }
+
+ private void assertRemoteSpan() {
+ SpanData data = waitSpan("RpcServer.process");
+ assertTrue(data.getParentSpanContext().isRemote());
+ assertEquals(SpanKind.SERVER, data.getKind());
+ }
+
+ @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());
+ assertRpcAttribute(waitSpan("RpcClient.callMethod"), "pause", rpcServer.getListenerAddress(),
+ SpanKind.CLIENT);
+ assertRpcAttribute(waitSpan("RpcServer.callMethod"), "pause", null, SpanKind.INTERNAL);
+ assertRemoteSpan();
+ 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()));
+ assertRpcAttribute(waitSpan("RpcClient.callMethod"), "error", rpcServer.getListenerAddress(),
+ SpanKind.CLIENT);
+ assertRpcAttribute(waitSpan("RpcServer.callMethod"), "error", null, SpanKind.INTERNAL);
+ assertRemoteSpan();
+ 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 2601fba05884..c3b52a97cc15 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.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/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 0707a53bcfde..04aa3dfbd868 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 MockHRegion(Path tableDir, WAL log, FileSystem fs, Configuration conf,
}
@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/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 9763841ff2d7..b4e0110f4992 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 @@ void setStoreFlushCtx(StoreFlushContext storeFlushCtx) {
}
@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/TestHRegionTracing.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
new file mode 100644
index 000000000000..e5779b43cbc6
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionTracing.java
@@ -0,0 +1,183 @@
+/**
+ * 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.fs.Path;
+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.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 WAL wal;
+
+ private HRegion region;
+
+ @AfterClass
+ public static void tearDownAfterClass() throws IOException {
+ 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);
+ 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);
+ }
+
+ @After
+ public void tearDown() throws IOException {
+ if (region != null) {
+ region.close();
+ }
+ Closeables.close(wal, true);
+ }
+
+ private void assertSpan(String spanName) {
+ assertTrue(traceRule.getSpans().stream().anyMatch(span -> {
+ if (!span.getName().equals(spanName)) {
+ return false;
+ }
+ List 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.close");
+ }
+}
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 a482d934e948..a2ac3375ab6e 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 @@ public void append(Entry entry) {
}
@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 3c250446bec9..78531f350c15 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 @@ public void append(Entry entry) throws IOException {
}
@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/trace/TestHTraceHooks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java
deleted file mode 100644
index b1fc5b98be3d..000000000000
--- 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() {
- @Override public boolean evaluate() throws Exception {
- return (rcvr == null) ? true : rcvr.getSpans().size() >= 5;
- }
- });
-
- Collection spans = Sets.newHashSet(rcvr.getSpans());
- List 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 eb209d0ee158..000000000000
--- 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 COMPARATOR =
- new Comparator() {
- @Override
- public int compare(Span a, Span b) {
- return a.getSpanId().compareTo(b.getSpanId());
- }
- };
-
- private final TreeSet treeSet;
-
- private final HashMap> parentToSpans;
-
- SpansByParent(Collection spans) {
- TreeSet treeSet = new TreeSet(COMPARATOR);
- parentToSpans = new HashMap>();
- for (Span span : spans) {
- treeSet.add(span);
- for (SpanId parent : span.getParents()) {
- LinkedList list = parentToSpans.get(parent);
- if (list == null) {
- list = new LinkedList();
- parentToSpans.put(parent, list);
- }
- list.add(span);
- }
- if (span.getParents().length == 0) {
- LinkedList list = parentToSpans.get(SpanId.INVALID);
- if (list == null) {
- list = new LinkedList();
- parentToSpans.put(SpanId.INVALID, list);
- }
- list.add(span);
- }
- }
- this.treeSet = treeSet;
- }
-
- public List find(SpanId parentId) {
- LinkedList spans = parentToSpans.get(parentId);
- if (spans == null) {
- return new LinkedList();
- }
- return spans;
- }
-
- public Iterator iterator() {
- return Collections.unmodifiableSortedSet(treeSet).iterator();
- }
- }
-
- public static class SpansByProcessId {
- private static Comparator COMPARATOR =
- new Comparator() {
- @Override
- public int compare(Span a, Span b) {
- return a.getSpanId().compareTo(b.getSpanId());
- }
- };
-
- private final TreeSet treeSet;
-
- SpansByProcessId(Collection spans) {
- TreeSet treeSet = new TreeSet(COMPARATOR);
- for (Span span : spans) {
- treeSet.add(span);
- }
- this.treeSet = treeSet;
- }
-
- public Iterator 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 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 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/FaultyFSLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/FaultyFSLog.java
index f58d36428cf4..3afafa16982e 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 void setFailureType(FailureType fType) {
}
@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
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 9f02d2e4ed92..6b6175115be3 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.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.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.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 @@ class WALPutBenchmark implements Runnable {
private final boolean noSync;
private final HRegion region;
private final int syncInterval;
- private final Sampler loopSampler;
private final NavigableMap 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 @@ class WALPutBenchmark implements Runnable {
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 void run() {
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 void run() {
}
}
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 int run(String[] args) throws Exception {
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 int run(String[] args) throws Exception {
} 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 int run(String[] args) throws Exception {
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 int run(String[] args) throws Exception {
// 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 int run(String[] args) throws Exception {
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 44c8bf6d77f7..2d06a9d487d8 100644
--- a/hbase-shaded/hbase-shaded-client/pom.xml
+++ b/hbase-shaded/hbase-shaded-client/pom.xml
@@ -79,6 +79,7 @@
log4j:*
commons-logging:*
org.javassist:*
+ io.opentelemetry:*
diff --git a/hbase-shaded/hbase-shaded-testing-util/pom.xml b/hbase-shaded/hbase-shaded-testing-util/pom.xml
index 6ee31e24dfa6..ac6ab4941b87 100644
--- a/hbase-shaded/hbase-shaded-testing-util/pom.xml
+++ b/hbase-shaded/hbase-shaded-testing-util/pom.xml
@@ -236,6 +236,7 @@
log4j:*
commons-logging:*
org.javassist:*
+ io.opentelemetry:*
diff --git a/hbase-shaded/pom.xml b/hbase-shaded/pom.xml
index 2a44f435d340..d1dfb06f3742 100644
--- a/hbase-shaded/pom.xml
+++ b/hbase-shaded/pom.xml
@@ -159,6 +159,7 @@
log4j:*
commons-logging:*
org.javassist:*
+ io.opentelemetry:*
diff --git a/hbase-shell/src/main/ruby/shell/commands/trace.rb b/hbase-shell/src/main/ruby/shell/commands/trace.rb
index f2a8ee0b086d..f52f474b8372 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 @@ def command(startstop = 'status', spanname = 'HBaseShell')
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 dd8ab248a767..181403181211 100644
--- a/hbase-zookeeper/pom.xml
+++ b/hbase-zookeeper/pom.xml
@@ -148,6 +148,10 @@
org.apache.zookeeper
zookeeper
| | | |
+
+ io.opentelemetry
+ opentelemetry-api
+
junit
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 90cb123c76f4..f5cc1b20d779 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 synchronized void reconnectAfterExpiration()
* 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 void delete(String path, int version) throws InterruptedException, Keeper
retryCounter.sleepUntilNextRetry();
isRetry = true;
}
+ } finally {
+ span.end();
}
}
@@ -204,7 +207,8 @@ public void delete(String path, int version) throws InterruptedException, Keeper
* @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 Stat exists(String path, Watcher watcher) throws KeeperException, Interru
}
retryCounter.sleepUntilNextRetry();
}
+ } finally {
+ span.end();
}
}
@@ -233,7 +239,9 @@ public Stat exists(String path, Watcher watcher) throws KeeperException, Interru
* @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 Stat exists(String path, boolean watch) throws KeeperException, Interrupt
}
retryCounter.sleepUntilNextRetry();
}
+ } finally {
+ span.end();
}
}
@@ -273,7 +283,9 @@ private void retryOrThrow(RetryCounter retryCounter, KeeperException e,
*/
public List 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 List getChildren(String path, Watcher watcher)
}
retryCounter.sleepUntilNextRetry();
}
+ } finally {
+ span.end();
}
}
@@ -303,7 +317,9 @@ public List getChildren(String path, Watcher watcher)
*/
public List 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 List getChildren(String path, boolean watch)
}
retryCounter.sleepUntilNextRetry();
}
+ } finally {
+ span.end();
}
}
@@ -334,7 +352,8 @@ public List getChildren(String path, boolean watch)
*/
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 byte[] getData(String path, Watcher watcher, Stat stat)
}
retryCounter.sleepUntilNextRetry();
}
+ } finally {
+ span.end();
}
}
@@ -364,7 +385,9 @@ public byte[] getData(String path, Watcher watcher, Stat stat)
*/
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 byte[] getData(String path, boolean watch, Stat stat)
}
retryCounter.sleepUntilNextRetry();
}
+ } finally {
+ span.end();
}
}
@@ -397,7 +422,8 @@ public byte[] getData(String path, boolean watch, Stat stat)
*/
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 Stat setData(String path, byte[] data, int version)
retryCounter.sleepUntilNextRetry();
isRetry = true;
}
+ } finally {
+ span.end();
}
}
@@ -444,9 +472,9 @@ public Stat setData(String path, byte[] data, int version)
* getAcl is an idempotent operation. Retry before throwing exception
* @return list of ACLs
*/
- public List getAcl(String path, Stat stat)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.getAcl")) {
+ public List 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 List getAcl(String path, Stat stat)
}
retryCounter.sleepUntilNextRetry();
}
+ } finally {
+ span.end();
}
}
@@ -476,7 +506,8 @@ public List getAcl(String path, Stat stat)
*/
public Stat setAcl(String path, List 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 Stat setAcl(String path, List acls, int version)
}
retryCounter.sleepUntilNextRetry();
}
+ } finally {
+ span.end();
}
}
@@ -514,10 +547,10 @@ public Stat setAcl(String path, List acls, int version)
*
* @return Path
*/
- public String create(String path, byte[] data, List acl,
- CreateMode createMode)
+ public String create(String path, byte[] data, List 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 String create(String path, byte[] data, List acl,
throw new IllegalArgumentException("Unrecognized CreateMode: " +
createMode);
}
+ } finally {
+ span.end();
}
}
@@ -647,9 +682,9 @@ private Iterable prepareZKMulti(Iterable ops) throws UnsupportedOperatio
/**
* Run multiple operations in a transactional manner. Retry before throwing exception
*/
- public List multi(Iterable ops)
- throws KeeperException, InterruptedException {
- try (TraceScope scope = TraceUtil.createTrace("RecoverableZookeeper.multi")) {
+ public List multi(Iterable ops) throws KeeperException, InterruptedException {
+ Span span = TraceUtil.getGlobalTracer().spanBuilder("RecoverableZookeeper.multi").startSpan();
+ try (Scope scope = span.makeCurrent()) {
RetryCounter retryCounter = retryCounterFactory.create();
Iterable multiOps = prepareZKMulti(ops);
while (true) {
@@ -671,6 +706,8 @@ public List multi(Iterable ops)
}
retryCounter.sleepUntilNextRetry();
}
+ } finally {
+ span.end();
}
}
diff --git a/pom.xml b/pom.xml
index 87e3b475d1c3..e6437de16782 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1025,6 +1025,25 @@
+
+ banned-htrace
+
+ enforce
+
+
+
+
+
+ org.apache.htrace:**
+
+
+ Use OpenTelemetry instead
+
+ false
+
+
+
+
check-aggregate-license
@@ -1134,9 +1153,10 @@
true
512
- Do not use htrace v3
+ Do not use htrace
org.htrace.**
+ org.apache.htrace.**
@@ -1462,7 +1482,8 @@
9.2.13.0
4.13
1.3
- 4.2.0-incubating
+ 1.0.1
+ 1.0.1
1.2.17
2.28.2
@@ -2154,9 +2175,22 @@
test
- org.apache.htrace
- htrace-core4
- ${htrace.version}
+ io.opentelemetry
+ opentelemetry-bom
+ ${opentelemetry.version}
+ pom
+ import
+
+
+ io.opentelemetry
+ opentelemetry-semconv
+ ${opentelemetry.version}-alpha
+
+
+ io.opentelemetry.javaagent
+ opentelemetry-javaagent
+ ${opentelemetry-javaagent.version}
+ all
com.lmax
diff --git a/src/main/asciidoc/_chapters/tracing.adoc b/src/main/asciidoc/_chapters/tracing.adoc
new file mode 100644
index 000000000000..cc9f09362ea3
--- /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://github.com/open-telemetry/opentelemetry-java-instrumentation[opentelemetry-java-instrumentation] for more details on how to properly configure opentelemetry instrumentation.
+
+== 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 tracing data instead of logging.
+
+=== 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 cf1568e7aa05..d4f9e3755a64 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 <> 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 e0df010283a1..c264b06e4c92 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