You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2019/02/24 23:26:57 UTC
[hadoop] branch trunk updated: HDDS-1152. Add trace information for
the client side of the datanode writes Contributed by Elek, Marton.
This is an automated email from the ASF dual-hosted git repository.
aengineer pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git
The following commit(s) were added to refs/heads/trunk by this push:
new 3aa0a57 HDDS-1152. Add trace information for the client side of the datanode writes Contributed by Elek, Marton.
3aa0a57 is described below
commit 3aa0a57ea0236a32343928b6c9c9d71c6cb5a020
Author: Anu Engineer <ae...@apache.org>
AuthorDate: Sun Feb 24 15:20:34 2019 -0800
HDDS-1152. Add trace information for the client side of the datanode writes
Contributed by Elek, Marton.
---
.../apache/hadoop/hdds/scm/XceiverClientGrpc.java | 26 +++++++++++++---------
.../apache/hadoop/hdds/scm/XceiverClientRatis.java | 26 ++++++++++++++--------
.../hadoop/ozone/freon/RandomKeyGenerator.java | 11 ++++++---
3 files changed, 41 insertions(+), 22 deletions(-)
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 6fa54a5..ddcf966 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.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(
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 f68b28f..0caf10d 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 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
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 eb25b33..650d253 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 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;
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org