diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index 6fa54a5827..ddcf9667c5 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -37,6 +37,9 @@ import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Time; + +import io.opentracing.Scope; +import io.opentracing.util.GlobalTracer; import org.apache.ratis.thirdparty.io.grpc.ManagedChannel; import org.apache.ratis.thirdparty.io.grpc.Status; import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts; @@ -288,17 +291,20 @@ private XceiverClientReply sendCommandWithRetry( public XceiverClientReply sendCommandAsync( ContainerCommandRequestProto request) throws IOException, ExecutionException, InterruptedException { - XceiverClientReply asyncReply = - sendCommandAsync(request, pipeline.getFirstNode()); - - // TODO : for now make this API sync in nature as async requests are - // served out of order over XceiverClientGrpc. This needs to be fixed - // if this API is to be used for I/O path. Currently, this is not - // used for Read/Write Operation but for tests. - if (!HddsUtils.isReadOnly(request)) { - asyncReply.getResponse().get(); + try (Scope scope = GlobalTracer.get() + .buildSpan("XceiverClientGrpc." + request.getCmdType().name()) + .startActive(true)) { + XceiverClientReply asyncReply = + sendCommandAsync(request, pipeline.getFirstNode()); + // TODO : for now make this API sync in nature as async requests are + // served out of order over XceiverClientGrpc. This needs to be fixed + // if this API is to be used for I/O path. Currently, this is not + // used for Read/Write Operation but for tests. + if (!HddsUtils.isReadOnly(request)) { + asyncReply.getResponse().get(); + } + return asyncReply; } - return asyncReply; } private XceiverClientReply sendCommandAsync( diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java index f68b28f911..0caf10d2af 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java @@ -22,6 +22,9 @@ import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.security.x509.SecurityConfig; + +import io.opentracing.Scope; +import io.opentracing.util.GlobalTracer; import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.protocol.RaftRetryFailureException; @@ -191,15 +194,20 @@ private RaftClient getClient() { private CompletableFuture sendRequestAsync( ContainerCommandRequestProto request) { - ContainerCommandRequestProto finalPayload = - ContainerCommandRequestProto.newBuilder(request) - .setTraceID(TracingUtil.exportCurrentSpan()) - .build(); - boolean isReadOnlyRequest = HddsUtils.isReadOnly(finalPayload); - ByteString byteString = finalPayload.toByteString(); - LOG.debug("sendCommandAsync {} {}", isReadOnlyRequest, finalPayload); - return isReadOnlyRequest ? getClient().sendReadOnlyAsync(() -> byteString) : - getClient().sendAsync(() -> byteString); + try (Scope scope = GlobalTracer.get() + .buildSpan("XceiverClientRatis." + request.getCmdType().name()) + .startActive(true)) { + ContainerCommandRequestProto finalPayload = + ContainerCommandRequestProto.newBuilder(request) + .setTraceID(TracingUtil.exportCurrentSpan()) + .build(); + boolean isReadOnlyRequest = HddsUtils.isReadOnly(finalPayload); + ByteString byteString = finalPayload.toByteString(); + LOG.debug("sendCommandAsync {} {}", isReadOnlyRequest, finalPayload); + return isReadOnlyRequest ? + getClient().sendReadOnlyAsync(() -> byteString) : + getClient().sendAsync(() -> byteString); + } } // gets the minimum log index replicated to all servers diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java index eb25b337b2..650d253b88 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java @@ -1,3 +1,4 @@ + /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with this @@ -614,9 +615,13 @@ public void run() { .update(keyCreationDuration); keyCreationTime.getAndAdd(keyCreationDuration); long keyWriteStart = System.nanoTime(); - os.write(keyValue); - os.write(randomValue); - os.close(); + try (Scope writeScope = GlobalTracer.get() + .buildSpan("writeKeyData") + .startActive(true)) { + os.write(keyValue); + os.write(randomValue); + os.close(); + } long keyWriteDuration = System.nanoTime() - keyWriteStart;