|
@@ -37,6 +37,9 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.util.Time;
|
|
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.ManagedChannel;
|
|
import org.apache.ratis.thirdparty.io.grpc.Status;
|
|
import org.apache.ratis.thirdparty.io.grpc.Status;
|
|
import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts;
|
|
import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts;
|
|
@@ -288,17 +291,20 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|
public XceiverClientReply sendCommandAsync(
|
|
public XceiverClientReply sendCommandAsync(
|
|
ContainerCommandRequestProto request)
|
|
ContainerCommandRequestProto request)
|
|
throws IOException, ExecutionException, InterruptedException {
|
|
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(
|
|
private XceiverClientReply sendCommandAsync(
|