소스 검색

HDDS-1152. Add trace information for the client side of the datanode writes
Contributed by Elek, Marton.

Anu Engineer 6 년 전
부모
커밋
3aa0a57ea0

+ 16 - 10
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java

@@ -37,6 +37,9 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
 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 @@ public class XceiverClientGrpc extends XceiverClientSpi {
   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(

+ 17 - 9
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java

@@ -22,6 +22,9 @@ import com.google.common.base.Preconditions;
 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 @@ public final class XceiverClientRatis extends XceiverClientSpi {
 
   private CompletableFuture<RaftClientReply> 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

+ 8 - 3
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 final class RandomKeyGenerator implements Callable<Void> {
                     .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;