浏览代码

HADOOP-17424. Replace HTrace with No-Op tracer (#2645)

Siyao Meng 4 年之前
父节点
当前提交
1a205cc3ad
共有 69 个文件被更改,包括 433 次插入1779 次删除
  1. 1 3
      hadoop-client-modules/hadoop-client-api/pom.xml
  2. 0 2
      hadoop-client-modules/hadoop-client-check-invariants/pom.xml
  3. 0 2
      hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml
  4. 1 4
      hadoop-client-modules/hadoop-client-minicluster/pom.xml
  5. 1 11
      hadoop-client-modules/hadoop-client-runtime/pom.xml
  6. 0 4
      hadoop-common-project/hadoop-common/pom.xml
  7. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java
  8. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
  9. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  10. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java
  11. 1 13
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsTracer.java
  12. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java
  13. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java
  14. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java
  15. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java
  16. 35 29
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  17. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java
  18. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java
  19. 26 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/NullTraceScope.java
  20. 17 15
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/Span.java
  21. 32 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanContext.java
  22. 0 64
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverInfo.java
  23. 0 212
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdmin.java
  24. 0 69
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdminProtocol.java
  25. 0 118
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdminProtocolServerSideTranslatorPB.java
  26. 0 105
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdminProtocolTranslatorPB.java
  27. 23 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceConfiguration.java
  28. 38 18
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceScope.java
  29. 12 40
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java
  30. 98 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/Tracer.java
  31. 0 100
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TracerConfigurationManager.java
  32. 7 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java
  33. 1 1
      hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto
  34. 0 11
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShell.java
  35. 0 110
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java
  36. 0 71
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java
  37. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  38. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
  39. 7 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  40. 16 17
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
  41. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
  42. 25 23
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
  43. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
  44. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
  45. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
  46. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java
  47. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReencryptionStatusIterator.java
  48. 9 15
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
  49. 15 12
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
  50. 3 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto
  51. 0 26
      hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/TestDFSPacket.java
  52. 0 4
      hadoop-hdfs-project/hadoop-hdfs/pom.xml
  53. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java
  54. 16 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java
  55. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
  56. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  57. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  58. 3 42
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  59. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  60. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java
  61. 1 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  62. 0 34
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  63. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  64. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java
  65. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
  66. 0 180
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java
  67. 0 223
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
  68. 0 108
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
  69. 0 12
      hadoop-project/pom.xml

+ 1 - 3
hadoop-client-modules/hadoop-client-api/pom.xml

@@ -126,9 +126,7 @@
                       <excludes>
                         <exclude>org/apache/hadoop/*</exclude>
                         <exclude>org/apache/hadoop/**/*</exclude>
-                        <!-- Our non-shaded htrace and logging libraries -->
-                        <exclude>org/apache/htrace/*</exclude>
-                        <exclude>org/apache/htrace/**/*</exclude>
+                        <!-- Our non-shaded logging libraries -->
                         <exclude>org/slf4j/*</exclude>
                         <exclude>org/slf4j/**/*</exclude>
                         <exclude>org/apache/commons/logging/*</exclude>

+ 0 - 2
hadoop-client-modules/hadoop-client-check-invariants/pom.xml

@@ -80,8 +80,6 @@
                          but enforcer still sees it.
                     -->
                     <exclude>org.apache.hadoop:hadoop-annotations</exclude>
-                    <!-- We leave HTrace as an unshaded dependnecy on purpose so that tracing within a JVM will work -->
-                    <exclude>org.apache.htrace:htrace-core4</exclude>
                     <!-- Leave slf4j unshaded so downstream users can configure logging. -->
                     <exclude>org.slf4j:slf4j-api</exclude>
                     <!-- Leave commons-logging unshaded so downstream users can configure logging. -->

+ 0 - 2
hadoop-client-modules/hadoop-client-check-test-invariants/pom.xml

@@ -84,8 +84,6 @@
                          but enforcer still sees it.
                     -->
                     <exclude>org.apache.hadoop:hadoop-annotations</exclude>
-                    <!-- We leave HTrace as an unshaded dependnecy on purpose so that tracing within a JVM will work -->
-                    <exclude>org.apache.htrace:htrace-core4</exclude>
                     <!-- Leave slf4j unshaded so downstream users can configure logging. -->
                     <exclude>org.slf4j:slf4j-api</exclude>
                     <!-- Leave commons-logging unshaded so downstream users can configure logging. -->

+ 1 - 4
hadoop-client-modules/hadoop-client-minicluster/pom.xml

@@ -672,7 +672,6 @@
                       <exclude>org.apache.hadoop:hadoop-client-api</exclude>
                       <exclude>org.apache.hadoop:hadoop-client-runtime</exclude>
                       <!-- Fine to expose our purposefully not-shaded deps as dependencies -->
-                      <exclude>org.apache.htrace:htrace-core4</exclude>
                       <exclude>org.slf4j:slf4j-api</exclude>
                       <exclude>commons-logging:commons-logging</exclude>
                       <exclude>junit:junit</exclude>
@@ -870,9 +869,7 @@
                       <excludes>
                         <exclude>org/apache/hadoop/*</exclude>
                         <exclude>org/apache/hadoop/**/*</exclude>
-                        <!-- Our non-shaded htrace and logging libraries -->
-                        <exclude>org/apache/htrace/*</exclude>
-                        <exclude>org/apache/htrace/**/*</exclude>
+                        <!-- Our non-shaded logging libraries -->
                         <exclude>org/slf4j/*</exclude>
                         <exclude>org/slf4j/**/*</exclude>
                         <exclude>org/apache/commons/logging/*</exclude>

+ 1 - 11
hadoop-client-modules/hadoop-client-runtime/pom.xml

@@ -75,15 +75,9 @@
     </dependency>
     <!-- Since hadoop-client is listed as optional, we have to list transitive
          dependencies that we still want to show up.
-         * HTrace
          * Slf4j API
          * commons-logging
       -->
-    <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
-      <scope>runtime</scope>
-    </dependency>
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
@@ -146,8 +140,6 @@
                     <excludes>
                       <!-- We need a filter that matches just those things that aer included in the api jar -->
                       <exclude>org.apache.hadoop:hadoop-client-api</exclude>
-                      <!-- Leave HTrace as an unshaded dependency on purpose, since a static class member is used to trace within a given JVM instance -->
-                      <exclude>org.apache.htrace:htrace-core4</exclude>
                       <!-- Leave slf4j unshaded so downstream users can configure logging. -->
                       <exclude>org.slf4j:slf4j-api</exclude>
                       <!-- Leave commons-logging unshaded so downstream users can configure logging. -->
@@ -250,9 +242,7 @@
                       <excludes>
                         <exclude>org/apache/hadoop/*</exclude>
                         <exclude>org/apache/hadoop/**/*</exclude>
-                        <!-- Our non-shaded htrace and logging libraries -->
-                        <exclude>org/apache/htrace/*</exclude>
-                        <exclude>org/apache/htrace/**/*</exclude>
+                        <!-- Our non-shaded logging libraries -->
                         <exclude>org/slf4j/*</exclude>
                         <exclude>org/slf4j/**/*</exclude>
                         <exclude>org/apache/commons/logging/*</exclude>

+ 0 - 4
hadoop-common-project/hadoop-common/pom.xml

@@ -286,10 +286,6 @@
       <scope>test</scope>
     </dependency>
 
-    <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
-    </dependency>
     <dependency>
       <groupId>org.apache.zookeeper</groupId>
       <artifactId>zookeeper</artifactId>

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java

@@ -21,7 +21,7 @@ package org.apache.hadoop.fs;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.core.TraceScope;
+import org.apache.hadoop.tracing.TraceScope;
 
 import java.io.IOException;
 import java.io.OutputStream;

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java

@@ -67,7 +67,7 @@ import org.apache.hadoop.util.ShutdownHookManager;
 
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -83,8 +83,8 @@ import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.htrace.core.Tracer;
-import org.apache.htrace.core.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
 
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java

@@ -35,8 +35,8 @@ import org.apache.hadoop.tracing.TraceUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 1 - 13
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsTracer.java

@@ -22,7 +22,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.tracing.TraceUtils;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Tracer;
 
 /**
  * Holds the HTrace Tracer used for FileSystem operations.
@@ -47,18 +47,6 @@ public final class FsTracer {
     return instance;
   }
 
-  @VisibleForTesting
-  public static synchronized void clear() {
-    if (instance == null) {
-      return;
-    }
-    try {
-      instance.close();
-    } finally {
-      instance = null;
-    }
-  }
-
   private FsTracer() {
   }
 }

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Globber.java

@@ -27,8 +27,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.util.DurationInfo;
 
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java

@@ -54,8 +54,8 @@ import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.concurrent.AsyncGet;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Span;
+import org.apache.hadoop.tracing.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java

@@ -39,8 +39,8 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.concurrent.AsyncGet;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java

@@ -35,8 +35,8 @@ import org.apache.hadoop.security.token.SecretManager;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.concurrent.AsyncGet;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 35 - 29
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -99,6 +99,7 @@ import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.Rpc
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcSaslProto;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcSaslProto.SaslAuth;
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcSaslProto.SaslState;
+import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RPCTraceInfoProto;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.SaslPropertiesResolver;
@@ -118,10 +119,11 @@ import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.ProtoUtil;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
-
+import org.apache.hadoop.tracing.Span;
+import org.apache.hadoop.tracing.SpanContext;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
+import org.apache.hadoop.tracing.TraceUtils;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.thirdparty.protobuf.ByteString;
@@ -783,7 +785,7 @@ public abstract class Server {
     private AtomicInteger responseWaitCount = new AtomicInteger(1);
     final RPC.RpcKind rpcKind;
     final byte[] clientId;
-    private final TraceScope traceScope; // the HTrace scope on the server side
+    private final Span span; // the trace span on the server side
     private final CallerContext callerContext; // the call context
     private boolean deferredResponse = false;
     private int priorityLevel;
@@ -798,7 +800,7 @@ public abstract class Server {
 
     Call(Call call) {
       this(call.callId, call.retryCount, call.rpcKind, call.clientId,
-          call.traceScope, call.callerContext);
+          call.span, call.callerContext);
     }
 
     Call(int id, int retryCount, RPC.RpcKind kind, byte[] clientId) {
@@ -812,14 +814,14 @@ public abstract class Server {
     }
 
     Call(int id, int retryCount, RPC.RpcKind kind, byte[] clientId,
-        TraceScope traceScope, CallerContext callerContext) {
+        Span span, CallerContext callerContext) {
       this.callId = id;
       this.retryCount = retryCount;
       this.timestampNanos = Time.monotonicNowNanos();
       this.responseTimestampNanos = timestampNanos;
       this.rpcKind = kind;
       this.clientId = clientId;
-      this.traceScope = traceScope;
+      this.span = span;
       this.callerContext = callerContext;
       this.clientStateId = Long.MIN_VALUE;
       this.isCallCoordinated = false;
@@ -988,8 +990,8 @@ public abstract class Server {
 
     RpcCall(Connection connection, int id, int retryCount,
         Writable param, RPC.RpcKind kind, byte[] clientId,
-        TraceScope traceScope, CallerContext context) {
-      super(id, retryCount, kind, clientId, traceScope, context);
+        Span span, CallerContext context) {
+      super(id, retryCount, kind, clientId, span, context);
       this.connection = connection;
       this.rpcRequest = param;
     }
@@ -2672,19 +2674,24 @@ public abstract class Server {
         throw new FatalRpcServerException(
             RpcErrorCodeProto.FATAL_DESERIALIZING_REQUEST, err);
       }
-        
-      TraceScope traceScope = null;
+
+      Span span = null;
       if (header.hasTraceInfo()) {
-        if (tracer != null) {
-          // If the incoming RPC included tracing info, always continue the
-          // trace
-          SpanId parentSpanId = new SpanId(
-              header.getTraceInfo().getTraceId(),
-              header.getTraceInfo().getParentId());
-          traceScope = tracer.newScope(
-              RpcClientUtil.toTraceName(rpcRequest.toString()),
-              parentSpanId);
-          traceScope.detach();
+        RPCTraceInfoProto traceInfoProto = header.getTraceInfo();
+        if (traceInfoProto.hasSpanContext()) {
+          if (tracer == null) {
+            setTracer(Tracer.curThreadTracer());
+          }
+          if (tracer != null) {
+            // If the incoming RPC included tracing info, always continue the
+            // trace
+            SpanContext spanCtx = TraceUtils.byteStringToSpanContext(
+                traceInfoProto.getSpanContext());
+            if (spanCtx != null) {
+              span = tracer.newSpan(
+                  RpcClientUtil.toTraceName(rpcRequest.toString()), spanCtx);
+            }
+          }
         }
       }
 
@@ -2700,7 +2707,7 @@ public abstract class Server {
       RpcCall call = new RpcCall(this, header.getCallId(),
           header.getRetryCount(), rpcRequest,
           ProtoUtil.convert(header.getRpcKind()),
-          header.getClientId().toByteArray(), traceScope, callerContext);
+          header.getClientId().toByteArray(), span, callerContext);
 
       // Save the priority level assignment by the scheduler
       call.setPriorityLevel(callQueue.getPriorityLevel(call));
@@ -2953,10 +2960,9 @@ public abstract class Server {
             LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
           }
           CurCall.set(call);
-          if (call.traceScope != null) {
-            call.traceScope.reattach();
-            traceScope = call.traceScope;
-            traceScope.getSpan().addTimelineAnnotation("called");
+          if (call.span != null) {
+            traceScope = tracer.activateSpan(call.span);
+            call.span.addTimelineAnnotation("called");
           }
           // always update the current call context
           CallerContext.setCurrent(call.callerContext);
@@ -2971,14 +2977,14 @@ public abstract class Server {
           if (running) {                          // unexpected -- log it
             LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
             if (traceScope != null) {
-              traceScope.getSpan().addTimelineAnnotation("unexpectedly interrupted: " +
+              traceScope.addTimelineAnnotation("unexpectedly interrupted: " +
                   StringUtils.stringifyException(e));
             }
           }
         } catch (Exception e) {
           LOG.info(Thread.currentThread().getName() + " caught an exception", e);
           if (traceScope != null) {
-            traceScope.getSpan().addTimelineAnnotation("Exception: " +
+            traceScope.addTimelineAnnotation("Exception: " +
                 StringUtils.stringifyException(e));
           }
         } finally {

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java

@@ -39,8 +39,8 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.*;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java

@@ -34,8 +34,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.thirdparty.com.google.common.base.Ticker;
 import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;

+ 26 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/NullTraceScope.java

@@ -0,0 +1,26 @@
+/**
+ * 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.tracing;
+
+public class NullTraceScope extends TraceScope {
+  public static final NullTraceScope INSTANCE = new NullTraceScope();
+
+  public NullTraceScope() {
+    super(null);
+  }
+}

+ 17 - 15
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverInfoBuilder.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/Span.java

@@ -17,26 +17,28 @@
  */
 package org.apache.hadoop.tracing;
 
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
+import java.io.Closeable;
 
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class SpanReceiverInfoBuilder {
-  private SpanReceiverInfo info;
+public class Span implements Closeable {
 
-  public SpanReceiverInfoBuilder(String className) {
-    info = new SpanReceiverInfo(0, className);
+  public Span() {
   }
 
-  public void addConfigurationPair(String key, String value) {
-    info.configPairs.add(new ConfigurationPair(key, value));
+  public Span addKVAnnotation(String key, String value) {
+    return this;
   }
 
-  public SpanReceiverInfo build() {
-    SpanReceiverInfo ret = info;
-    info = null;
-    return ret;
+  public Span addTimelineAnnotation(String msg) {
+    return this;
+  }
+
+  public SpanContext getContext() {
+    return null;
+  }
+
+  public void finish() {
+  }
+
+  public void close() {
   }
 }

+ 32 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanContext.java

@@ -0,0 +1,32 @@
+/**
+ * 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.tracing;
+
+import java.io.Closeable;
+
+/**
+ * Wrapper class for SpanContext to avoid using OpenTracing/OpenTelemetry
+ * SpanContext class directly for better separation.
+ */
+public class SpanContext implements Closeable {
+  public SpanContext() {
+  }
+
+  public void close() {
+  }
+}

+ 0 - 64
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/SpanReceiverInfo.java

@@ -1,64 +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.tracing;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class SpanReceiverInfo {
-  private final long id;
-  private final String className;
-  final List<ConfigurationPair> configPairs =
-      new ArrayList<ConfigurationPair>();
-
-  static class ConfigurationPair {
-    private final String key;
-    private final String value;
-
-    ConfigurationPair(String key, String value) {
-      this.key = key;
-      this.value = value;
-    }
-
-    public String getKey() {
-      return key;
-    }
-
-    public String getValue() {
-      return value;
-    }
-  }
-
-  SpanReceiverInfo(long id, String className) {
-    this.id = id;
-    this.className = className;
-  }
-
-  public long getId() {
-    return id;
-  }
-
-  public String getClassName() {
-    return className;
-  }
-}

+ 0 - 212
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdmin.java

@@ -1,212 +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.tracing;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.InetSocketAddress;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.ipc.ProtobufRpcEngine2;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.tools.TableListing;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Tool;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A command-line tool for viewing and modifying tracing settings.
- */
-@InterfaceAudience.Private
-public class TraceAdmin extends Configured implements Tool {
-  private TraceAdminProtocolPB proxy;
-  private TraceAdminProtocolTranslatorPB remote;
-  private static final Logger LOG = LoggerFactory.getLogger(TraceAdmin.class);
-
-  private void usage() {
-    PrintStream err = System.err;
-    err.print(
-        "Hadoop tracing configuration commands:\n" +
-            "  -add [-class classname] [-Ckey=value] [-Ckey2=value2] ...\n" +
-            "    Add a span receiver with the provided class name.  Configuration\n" +
-            "    keys for the span receiver can be specified with the -C options.\n" +
-            "    The span receiver will also inherit whatever configuration keys\n" +
-            "    exist in the daemon's configuration.\n" +
-            "  -help: Print this help message.\n" +
-            "  -host [hostname:port]\n" +
-            "    Specify the hostname and port of the daemon to examine.\n" +
-            "    Required for all commands.\n" +
-            "  -list: List the current span receivers.\n" +
-            "  -remove [id]\n" +
-            "    Remove the span receiver with the specified id.  Use -list to\n" +
-            "    find the id of each receiver.\n" +
-            "  -principal: If the daemon is Kerberized, specify the service\n" +
-            "    principal name."
-    );
-  }
-
-  private int listSpanReceivers(List<String> args) throws IOException {
-    SpanReceiverInfo infos[] = remote.listSpanReceivers();
-    if (infos.length == 0) {
-      System.out.println("[no span receivers found]");
-      return 0;
-    }
-    TableListing listing = new TableListing.Builder().
-        addField("ID").
-        addField("CLASS").
-        showHeaders().
-        build();
-    for (SpanReceiverInfo info : infos) {
-      listing.addRow("" + info.getId(), info.getClassName());
-    }
-    System.out.println(listing.toString());
-    return 0;
-  }
-
-  private final static String CONFIG_PREFIX = "-C";
-
-  private int addSpanReceiver(List<String> args) throws IOException {
-    String className = StringUtils.popOptionWithArgument("-class", args);
-    if (className == null) {
-      System.err.println("You must specify the classname with -class.");
-      return 1;
-    }
-    ByteArrayOutputStream configStream = new ByteArrayOutputStream();
-    PrintStream configsOut = new PrintStream(configStream, false, "UTF-8");
-    SpanReceiverInfoBuilder factory = new SpanReceiverInfoBuilder(className);
-    String prefix = "";
-    for (int i = 0; i < args.size(); ++i) {
-      String str = args.get(i);
-      if (!str.startsWith(CONFIG_PREFIX)) {
-        System.err.println("Can't understand argument: " + str);
-        return 1;
-      }
-      str = str.substring(CONFIG_PREFIX.length());
-      int equalsIndex = str.indexOf("=");
-      if (equalsIndex < 0) {
-        System.err.println("Can't parse configuration argument " + str);
-        System.err.println("Arguments must be in the form key=value");
-        return 1;
-      }
-      String key = str.substring(0, equalsIndex);
-      String value = str.substring(equalsIndex + 1);
-      factory.addConfigurationPair(key, value);
-      configsOut.print(prefix + key + " = " + value);
-      prefix = ", ";
-    }
-
-    String configStreamStr = configStream.toString("UTF-8");
-    try {
-      long id = remote.addSpanReceiver(factory.build());
-      System.out.println("Added trace span receiver " + id +
-          " with configuration " + configStreamStr);
-    } catch (IOException e) {
-      System.out.println("addSpanReceiver error with configuration " +
-                             configStreamStr);
-      throw e;
-    }
-    return 0;
-  }
-
-  private int removeSpanReceiver(List<String> args) throws IOException {
-    String indexStr = StringUtils.popFirstNonOption(args);
-    long id = -1;
-    try {
-      id = Long.parseLong(indexStr);
-    } catch (NumberFormatException e) {
-      System.err.println("Failed to parse ID string " +
-          indexStr + ": " + e.getMessage());
-      return 1;
-    }
-    remote.removeSpanReceiver(id);
-    System.err.println("Removed trace span receiver " + id);
-    return 0;
-  }
-
-  @Override
-  public int run(String argv[]) throws Exception {
-    LinkedList<String> args = new LinkedList<String>();
-    for (String arg : argv) {
-      args.add(arg);
-    }
-    if (StringUtils.popOption("-h", args) ||
-        StringUtils.popOption("-help", args)) {
-      usage();
-      return 0;
-    } else if (args.size() == 0) {
-      usage();
-      return 0;
-    }
-    String hostPort = StringUtils.popOptionWithArgument("-host", args);
-    if (hostPort == null) {
-      System.err.println("You must specify a host with -host.");
-      return 1;
-    }
-    if (args.isEmpty()) {
-      System.err.println("You must specify an operation.");
-      return 1;
-    }
-    String servicePrincipal = StringUtils.popOptionWithArgument("-principal",
-        args);
-    if (servicePrincipal != null) {
-      LOG.debug("Set service principal: {}", servicePrincipal);
-      getConf().set(
-          CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY,
-          servicePrincipal);
-    }
-    RPC.setProtocolEngine(getConf(), TraceAdminProtocolPB.class,
-        ProtobufRpcEngine2.class);
-    InetSocketAddress address = NetUtils.createSocketAddr(hostPort);
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    Class<?> xface = TraceAdminProtocolPB.class;
-    proxy = (TraceAdminProtocolPB)RPC.getProxy(xface,
-        RPC.getProtocolVersion(xface), address,
-        ugi, getConf(), NetUtils.getDefaultSocketFactory(getConf()), 0);
-    remote = new TraceAdminProtocolTranslatorPB(proxy);
-    try {
-      if (args.get(0).equals("-list")) {
-        return listSpanReceivers(args.subList(1, args.size()));
-      } else if (args.get(0).equals("-add")) {
-        return addSpanReceiver(args.subList(1, args.size()));
-      } else if (args.get(0).equals("-remove")) {
-        return removeSpanReceiver(args.subList(1, args.size()));
-      } else {
-        System.err.println("Unrecognized tracing command: " + args.get(0));
-        System.err.println("Use -help for help.");
-        return 1;
-      }
-    } finally {
-      remote.close();
-    }
-  }
-
-  public static void main(String[] argv) throws Exception {
-    TraceAdmin admin = new TraceAdmin();
-    admin.setConf(new Configuration());
-    System.exit(admin.run(argv));
-  }
-}

+ 0 - 69
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdminProtocol.java

@@ -1,69 +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.tracing;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.retry.AtMostOnce;
-import org.apache.hadoop.io.retry.Idempotent;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.security.KerberosInfo;
-
-/**
- * Protocol interface that provides tracing.
- */
-@KerberosInfo(
-    serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public interface TraceAdminProtocol {
-  public static final long versionID = 1L;
-
-  /**
-   * List the currently active trace span receivers.
-   * 
-   * @throws IOException        On error.
-   */
-  @Idempotent
-  public SpanReceiverInfo[] listSpanReceivers() throws IOException;
-
-  /**
-   * Add a new trace span receiver.
-   * 
-   * @param desc                The span receiver description.
-   * @return                    The ID of the new trace span receiver.
-   *
-   * @throws IOException        On error.
-   */
-  @AtMostOnce
-  public long addSpanReceiver(SpanReceiverInfo desc) throws IOException;
-
-  /**
-   * Remove a trace span receiver.
-   *
-   * @param spanReceiverId      The id of the span receiver to remove.
-   * @throws IOException        On error.
-   */
-  @AtMostOnce
-  public void removeSpanReceiver(long spanReceiverId) throws IOException;
-}

+ 0 - 118
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdminProtocolServerSideTranslatorPB.java

@@ -1,118 +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.tracing;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.retry.AtMostOnce;
-import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.tracing.TraceAdminPB.AddSpanReceiverRequestProto;
-import org.apache.hadoop.tracing.TraceAdminPB.AddSpanReceiverResponseProto;
-import org.apache.hadoop.tracing.TraceAdminPB.ListSpanReceiversRequestProto;
-import org.apache.hadoop.tracing.TraceAdminPB.ListSpanReceiversResponseProto;
-import org.apache.hadoop.tracing.TraceAdminPB.ConfigPair;
-import org.apache.hadoop.tracing.TraceAdminPB.RemoveSpanReceiverRequestProto;
-import org.apache.hadoop.tracing.TraceAdminPB.RemoveSpanReceiverResponseProto;
-import org.apache.hadoop.thirdparty.protobuf.RpcController;
-import org.apache.hadoop.thirdparty.protobuf.ServiceException;
-
-@InterfaceAudience.Private
-public class TraceAdminProtocolServerSideTranslatorPB
-    implements TraceAdminProtocolPB, Closeable  {
-  private final TraceAdminProtocol server;
-
-  public TraceAdminProtocolServerSideTranslatorPB(TraceAdminProtocol server) {
-    this.server = server;
-  }
-
-  @Override
-  public void close() throws IOException {
-    RPC.stopProxy(server);
-  }
-
-  @Override
-  public ListSpanReceiversResponseProto listSpanReceivers(
-      RpcController controller, ListSpanReceiversRequestProto req)
-          throws ServiceException {
-    try {
-      SpanReceiverInfo[] descs = server.listSpanReceivers();
-      ListSpanReceiversResponseProto.Builder bld =
-          ListSpanReceiversResponseProto.newBuilder();
-      for (int i = 0; i < descs.length; ++i) {
-        bld.addDescriptions(TraceAdminPB.SpanReceiverListInfo.newBuilder().
-              setId(descs[i].getId()).
-              setClassName(descs[i].getClassName()).build());
-      }
-      return bld.build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
-  @Override
-  public AddSpanReceiverResponseProto addSpanReceiver(
-      RpcController controller, AddSpanReceiverRequestProto req)
-          throws ServiceException {
-    try {
-      SpanReceiverInfoBuilder factory =
-        new SpanReceiverInfoBuilder(req.getClassName());
-      for (ConfigPair config : req.getConfigList()) {
-        factory.addConfigurationPair(config.getKey(), config.getValue());
-      }
-      long id = server.addSpanReceiver(factory.build());
-      return AddSpanReceiverResponseProto.newBuilder().setId(id).build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
-  @Override
-  public RemoveSpanReceiverResponseProto removeSpanReceiver(
-      RpcController controller,  RemoveSpanReceiverRequestProto req)
-          throws ServiceException {
-    try {
-      server.removeSpanReceiver(req.getId());
-      return RemoveSpanReceiverResponseProto.getDefaultInstance();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    return TraceAdminProtocol.versionID;
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-          long clientVersion, int clientMethodsHash) throws IOException {
-    if (!protocol.equals(RPC.getProtocolName(TraceAdminProtocolPB.class))) {
-      throw new IOException("Serverside implements " +
-          RPC.getProtocolName(TraceAdminProtocolPB.class) +
-          ". The following requested protocol is unknown: " + protocol);
-    }
-    return ProtocolSignature.getProtocolSignature(clientMethodsHash,
-        RPC.getProtocolVersion(TraceAdminProtocolPB.class),
-        TraceAdminProtocolPB.class);
-  }
-}

+ 0 - 105
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdminProtocolTranslatorPB.java

@@ -1,105 +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.tracing;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.ipc.ProtobufHelper;
-import org.apache.hadoop.ipc.ProtocolTranslator;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.tracing.TraceAdminPB.AddSpanReceiverRequestProto;
-import org.apache.hadoop.tracing.TraceAdminPB.AddSpanReceiverResponseProto;
-import org.apache.hadoop.tracing.TraceAdminPB.ListSpanReceiversRequestProto;
-import org.apache.hadoop.tracing.TraceAdminPB.ListSpanReceiversResponseProto;
-import org.apache.hadoop.tracing.TraceAdminPB.ConfigPair;
-import org.apache.hadoop.tracing.TraceAdminPB.RemoveSpanReceiverRequestProto;
-import org.apache.hadoop.tracing.TraceAdminPB.SpanReceiverListInfo;
-import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
-import org.apache.hadoop.thirdparty.protobuf.ServiceException;
-
-@InterfaceAudience.Private
-public class TraceAdminProtocolTranslatorPB implements
-    TraceAdminProtocol, ProtocolTranslator, Closeable  {
-  private final TraceAdminProtocolPB rpcProxy;
-
-  public TraceAdminProtocolTranslatorPB(TraceAdminProtocolPB rpcProxy) {
-    this.rpcProxy = rpcProxy;
-  }
-
-  @Override
-  public void close() throws IOException {
-    RPC.stopProxy(rpcProxy);
-  }
-
-  @Override
-  public SpanReceiverInfo[] listSpanReceivers() throws IOException {
-    ArrayList<SpanReceiverInfo> infos = new ArrayList<SpanReceiverInfo>(1);
-    try {
-      ListSpanReceiversRequestProto req =
-          ListSpanReceiversRequestProto.newBuilder().build();
-      ListSpanReceiversResponseProto resp =
-          rpcProxy.listSpanReceivers(null, req);
-      for (SpanReceiverListInfo info : resp.getDescriptionsList()) {
-        infos.add(new SpanReceiverInfo(info.getId(), info.getClassName()));
-      }
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-    return infos.toArray(new SpanReceiverInfo[infos.size()]);
-  }
-
-  @Override
-  public long addSpanReceiver(SpanReceiverInfo info) throws IOException {
-    try {
-      AddSpanReceiverRequestProto.Builder bld =
-          AddSpanReceiverRequestProto.newBuilder();
-      bld.setClassName(info.getClassName());
-      for (ConfigurationPair configPair : info.configPairs) {
-        ConfigPair tuple = ConfigPair.newBuilder().
-            setKey(configPair.getKey()).
-            setValue(configPair.getValue()).build();
-        bld.addConfig(tuple);
-      }
-      AddSpanReceiverResponseProto resp =
-          rpcProxy.addSpanReceiver(null, bld.build());
-      return resp.getId();
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public void removeSpanReceiver(long spanReceiverId) throws IOException {
-    try {
-      RemoveSpanReceiverRequestProto req =
-          RemoveSpanReceiverRequestProto.newBuilder()
-            .setId(spanReceiverId).build();
-      rpcProxy.removeSpanReceiver(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
-  @Override
-  public Object getUnderlyingProxyObject() {
-    return rpcProxy;
-  }
-}

+ 23 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceConfiguration.java

@@ -0,0 +1,23 @@
+/**
+ * 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.tracing;
+
+public class TraceConfiguration {
+  public TraceConfiguration() {
+  }
+}

+ 38 - 18
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceAdminProtocolPB.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceScope.java

@@ -15,24 +15,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.hadoop.tracing;
 
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.hadoop.tracing.TraceAdminPB.TraceAdminService;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.security.KerberosInfo;
-
-@KerberosInfo(
-    serverPrincipal=CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_USER_NAME_KEY)
-@ProtocolInfo(
-    protocolName = "org.apache.hadoop.tracing.TraceAdminPB.TraceAdminService",
-    protocolVersion = 1)
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public interface TraceAdminProtocolPB extends
-    TraceAdminService.BlockingInterface, VersionedProtocol {
+import java.io.Closeable;
+
+public class TraceScope implements Closeable {
+  Span span;
+
+  public TraceScope(Span span) {
+    this.span = span;
+  }
+
+  // Add tag to the span
+  public void addKVAnnotation(String key, String value) {
+  }
+
+  public void addKVAnnotation(String key, Number value) {
+  }
+
+  public void addTimelineAnnotation(String msg) {
+  }
+
+  public Span span() {
+    return span;
+  }
+
+  public Span getSpan() {
+    return span;
+  }
+
+  public void reattach() {
+  }
+
+  public void detach() {
+  }
+
+  public void close() {
+    if (span != null) {
+      span.close();
+    }
+  }
 }

+ 12 - 40
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TraceUtils.java

@@ -17,59 +17,31 @@
  */
 package org.apache.hadoop.tracing;
 
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
-import org.apache.htrace.core.HTraceConfiguration;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
 
 /**
  * This class provides utility functions for tracing.
  */
 @InterfaceAudience.Private
 public class TraceUtils {
-  private static List<ConfigurationPair> EMPTY = Collections.emptyList();
   static final String DEFAULT_HADOOP_TRACE_PREFIX = "hadoop.htrace.";
 
-  public static HTraceConfiguration wrapHadoopConf(final String prefix,
-        final Configuration conf) {
-    return wrapHadoopConf(prefix, conf, EMPTY);
+  public static TraceConfiguration wrapHadoopConf(final String prefix,
+      final Configuration conf) {
+    return null;
   }
 
-  public static HTraceConfiguration wrapHadoopConf(final String prefix,
-        final Configuration conf, List<ConfigurationPair> extraConfig) {
-    final HashMap<String, String> extraMap = new HashMap<String, String>();
-    for (ConfigurationPair pair : extraConfig) {
-      extraMap.put(pair.getKey(), pair.getValue());
-    }
-    return new HTraceConfiguration() {
-      @Override
-      public String get(String key) {
-        String ret = getInternal(prefix + key);
-        if (ret != null) {
-          return ret;
-        }
-        return getInternal(DEFAULT_HADOOP_TRACE_PREFIX  + key);
-      }
+  public static Tracer createAndRegisterTracer(String name) {
+    return null;
+  }
 
-      @Override
-      public String get(String key, String defaultValue) {
-        String ret = get(key);
-        if (ret != null) {
-          return ret;
-        }
-        return defaultValue;
-      }
+  public static SpanContext byteStringToSpanContext(ByteString byteString) {
+    return null;
+  }
 
-      private String getInternal(String key) {
-        if (extraMap.containsKey(key)) {
-          return extraMap.get(key);
-        }
-        return conf.get(key);
-      }
-    };
+  public static ByteString spanContextToByteString(SpanContext context) {
+    return null;
   }
 }

+ 98 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/Tracer.java

@@ -0,0 +1,98 @@
+/**
+ * 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.tracing;
+
+/**
+ * No-Op Tracer (for now) to remove HTrace without changing too many files.
+ */
+public class Tracer {
+  // Singleton
+  private static final Tracer globalTracer = null;
+  private final NullTraceScope nullTraceScope;
+  private final String name;
+
+  public final static String SPAN_RECEIVER_CLASSES_KEY =
+      "span.receiver.classes";
+
+  public Tracer(String name) {
+    this.name = name;
+    nullTraceScope = NullTraceScope.INSTANCE;
+  }
+
+  // Keeping this function at the moment for HTrace compatiblity,
+  // in fact all threads share a single global tracer for OpenTracing.
+  public static Tracer curThreadTracer() {
+    return globalTracer;
+  }
+
+  /***
+   * Return active span.
+   * @return org.apache.hadoop.tracing.Span
+   */
+  public static Span getCurrentSpan() {
+    return null;
+  }
+
+  public TraceScope newScope(String description) {
+    return nullTraceScope;
+  }
+
+  public Span newSpan(String description, SpanContext spanCtx) {
+    return new Span();
+  }
+
+  public TraceScope newScope(String description, SpanContext spanCtx) {
+    return nullTraceScope;
+  }
+
+  public TraceScope newScope(String description, SpanContext spanCtx,
+      boolean finishSpanOnClose) {
+    return nullTraceScope;
+  }
+
+  public TraceScope activateSpan(Span span) {
+    return nullTraceScope;
+  }
+
+  public void close() {
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public static class Builder {
+    static Tracer globalTracer;
+    private String name;
+
+    public Builder(final String name) {
+      this.name = name;
+    }
+
+    public Builder conf(TraceConfiguration conf) {
+      return this;
+    }
+
+    public Tracer build() {
+      if (globalTracer == null) {
+        globalTracer = new Tracer(name);
+      }
+      return globalTracer;
+    }
+  }
+}

+ 0 - 100
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tracing/TracerConfigurationManager.java

@@ -1,100 +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.tracing;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
-import org.apache.htrace.core.SpanReceiver;
-import org.apache.htrace.core.TracerPool;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class provides functions for managing the tracer configuration at
- * runtime via an RPC protocol.
- */
-@InterfaceAudience.Private
-public class TracerConfigurationManager implements TraceAdminProtocol {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(TracerConfigurationManager.class);
-
-  private final String confPrefix;
-  private final Configuration conf;
-
-  public TracerConfigurationManager(String confPrefix, Configuration conf) {
-    this.confPrefix = confPrefix;
-    this.conf = conf;
-  }
-
-  public synchronized SpanReceiverInfo[] listSpanReceivers()
-      throws IOException {
-    TracerPool pool = TracerPool.getGlobalTracerPool();
-    SpanReceiver[] receivers = pool.getReceivers();
-    SpanReceiverInfo[] info = new SpanReceiverInfo[receivers.length];
-    for (int i = 0; i < receivers.length; i++) {
-      SpanReceiver receiver = receivers[i];
-      info[i] = new SpanReceiverInfo(receiver.getId(),
-          receiver.getClass().getName());
-    }
-    return info;
-  }
-
-  public synchronized long addSpanReceiver(SpanReceiverInfo info)
-      throws IOException {
-    StringBuilder configStringBuilder = new StringBuilder();
-    String prefix = "";
-    for (ConfigurationPair pair : info.configPairs) {
-      configStringBuilder.append(prefix).append(pair.getKey()).
-          append(" = ").append(pair.getValue());
-      prefix = ", ";
-    }
-    SpanReceiver rcvr = null;
-    try {
-      rcvr = new SpanReceiver.Builder(TraceUtils.wrapHadoopConf(
-              confPrefix, conf, info.configPairs)).
-          className(info.getClassName().trim()).
-          build();
-    } catch (RuntimeException e) {
-      LOG.info("Failed to add SpanReceiver " + info.getClassName() +
-          " with configuration " + configStringBuilder.toString(), e);
-      throw e;
-    }
-    TracerPool.getGlobalTracerPool().addReceiver(rcvr);
-    LOG.info("Successfully added SpanReceiver " + info.getClassName() +
-        " with configuration " + configStringBuilder.toString());
-    return rcvr.getId();
-  }
-
-  public synchronized void removeSpanReceiver(long spanReceiverId)
-      throws IOException {
-    SpanReceiver[] receivers =
-        TracerPool.getGlobalTracerPool().getReceivers();
-    for (SpanReceiver receiver : receivers) {
-      if (receiver.getId() == spanReceiverId) {
-        TracerPool.getGlobalTracerPool().removeAndCloseReceiver(receiver);
-        LOG.info("Successfully removed SpanReceiver " + spanReceiverId +
-            " with class " + receiver.getClass().getName());
-        return;
-      }
-    }
-    throw new IOException("There is no span receiver with id " + spanReceiverId);
-  }
-}

+ 7 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java

@@ -29,8 +29,9 @@ import org.apache.hadoop.ipc.protobuf.IpcConnectionContextProtos.UserInformation
 import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.*;
 import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Span;
+import org.apache.hadoop.tracing.Tracer;
+import org.apache.hadoop.tracing.TraceUtils;
 
 import org.apache.hadoop.thirdparty.protobuf.ByteString;
 
@@ -180,10 +181,10 @@ public abstract class ProtoUtil {
     // Add tracing info if we are currently tracing.
     Span span = Tracer.getCurrentSpan();
     if (span != null) {
-      result.setTraceInfo(RPCTraceInfoProto.newBuilder()
-          .setTraceId(span.getSpanId().getHigh())
-          .setParentId(span.getSpanId().getLow())
-            .build());
+      RPCTraceInfoProto.Builder traceInfoProtoBuilder =
+          RPCTraceInfoProto.newBuilder().setSpanContext(
+              TraceUtils.spanContextToByteString(span.getContext()));
+      result.setTraceInfo(traceInfoProtoBuilder);
     }
 
     // Add caller context if it is not null

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/proto/RpcHeader.proto

@@ -63,7 +63,7 @@ enum RpcKindProto {
 message RPCTraceInfoProto {
     optional int64 traceId = 1; // parentIdHigh
     optional int64 parentId = 2; // parentIdLow
-
+    optional bytes spanContext = 3; // Trace SpanContext
 }
 
 /**

+ 0 - 11
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFsShell.java

@@ -21,10 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.shell.Command;
 import org.apache.hadoop.fs.shell.CommandFactory;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.tracing.SetSpanReceiver;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.htrace.core.AlwaysSampler;
-import org.apache.htrace.core.Tracer;
 import org.hamcrest.core.StringContains;
 import org.junit.Assert;
 import org.junit.Test;
@@ -53,10 +50,6 @@ public class TestFsShell {
   public void testTracing() throws Throwable {
     Configuration conf = new Configuration();
     String prefix = "fs.shell.htrace.";
-    conf.set(prefix + Tracer.SPAN_RECEIVER_CLASSES_KEY,
-        SetSpanReceiver.class.getName());
-    conf.set(prefix + Tracer.SAMPLER_CLASSES_KEY,
-        AlwaysSampler.class.getName());
     conf.setQuietMode(false);
     FsShell shell = new FsShell(conf);
     int res;
@@ -65,10 +58,6 @@ public class TestFsShell {
     } finally {
       shell.close();
     }
-    SetSpanReceiver.assertSpanNamesFound(new String[]{"help"});
-    Assert.assertEquals("-help ls cat",
-        SetSpanReceiver.getMap()
-            .get("help").get(0).getKVAnnotations().get("args"));
   }
 
   @Test

+ 0 - 110
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java

@@ -1,110 +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.tracing;
-
-import java.util.function.Supplier;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.SpanReceiver;
-import org.apache.htrace.core.HTraceConfiguration;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeoutException;
-import org.junit.Assert;
-
-/**
- * Span receiver that puts all spans into a single set.
- * This is useful for testing.
- * <p>
- * We're not using HTrace's POJOReceiver here so as that doesn't
- * push all the metrics to a static place, and would make testing
- * SpanReceiverHost harder.
- */
-public class SetSpanReceiver extends SpanReceiver {
-
-  public SetSpanReceiver(HTraceConfiguration conf) {
-  }
-
-  public void receiveSpan(Span span) {
-    SetHolder.spans.put(span.getSpanId(), span);
-  }
-
-  public void close() {
-  }
-
-  public static void clear() {
-    SetHolder.spans.clear();
-  }
-
-  public static int size() {
-    return SetHolder.spans.size();
-  }
-
-  public static Collection<Span> getSpans() {
-    return SetHolder.spans.values();
-  }
-
-  public static Map<String, List<Span>> getMap() {
-    return SetHolder.getMap();
-  }
-
-  public static class SetHolder {
-    public static ConcurrentHashMap<SpanId, Span> spans =
-        new ConcurrentHashMap<SpanId, Span>();
-
-    public static Map<String, List<Span>> getMap() {
-      Map<String, List<Span>> map = new HashMap<String, List<Span>>();
-
-      for (Span s : spans.values()) {
-        List<Span> l = map.get(s.getDescription());
-        if (l == null) {
-          l = new LinkedList<Span>();
-          map.put(s.getDescription(), l);
-        }
-        l.add(s);
-      }
-      return map;
-    }
-  }
-
-  public static void assertSpanNamesFound(final String[] expectedSpanNames) {
-    try {
-      GenericTestUtils.waitFor(new Supplier<Boolean>() {
-        @Override
-        public Boolean get() {
-          Map<String, List<Span>> map = SetSpanReceiver.SetHolder.getMap();
-          for (String spanName : expectedSpanNames) {
-            if (!map.containsKey(spanName)) {
-              return false;
-            }
-          }
-          return true;
-        }
-      }, 100, 1000);
-    } catch (TimeoutException e) {
-      Assert.fail("timed out to get expected spans: " + e.getMessage());
-    } catch (InterruptedException e) {
-      Assert.fail("interrupted while waiting spans: " + e.getMessage());
-    }
-  }
-}

+ 0 - 71
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/TestTraceUtils.java

@@ -1,71 +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.tracing;
-
-import static org.junit.Assert.assertEquals;
-
-import java.net.URI;
-import java.util.LinkedList;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.tracing.SpanReceiverInfo.ConfigurationPair;
-import org.apache.htrace.core.HTraceConfiguration;
-import org.junit.Test;
-
-public class TestTraceUtils {
-  private static String TEST_PREFIX = "test.prefix.htrace.";
-
-  @Test
-  public void testWrappedHadoopConf() {
-    String key = "sampler";
-    String value = "ProbabilitySampler";
-    Configuration conf = new Configuration();
-    conf.set(TEST_PREFIX + key, value);
-    HTraceConfiguration wrapped = TraceUtils.wrapHadoopConf(TEST_PREFIX, conf);
-    assertEquals(value, wrapped.get(key));
-  }
-
-  @Test
-  public void testExtraConfig() {
-    String key = "test.extra.config";
-    String oldValue = "old value";
-    String newValue = "new value";
-    Configuration conf = new Configuration();
-    conf.set(TEST_PREFIX + key, oldValue);
-    LinkedList<ConfigurationPair> extraConfig =
-        new LinkedList<ConfigurationPair>();
-    extraConfig.add(new ConfigurationPair(TEST_PREFIX + key, newValue));
-    HTraceConfiguration wrapped = TraceUtils.wrapHadoopConf(TEST_PREFIX, conf, extraConfig);
-    assertEquals(newValue, wrapped.get(key));
-  }
-
-  /**
-   * Test tracing the globber.  This is a regression test for HDFS-9187.
-   */
-  @Test
-  public void testTracingGlobber() throws Exception {
-    // Bypass the normal FileSystem object creation path by just creating an
-    // instance of a subclass.
-    FileSystem fs = new LocalFileSystem();
-    fs.initialize(new URI("file:///"), new Configuration());
-    fs.globStatus(new Path("/"));
-    fs.close();
-  }
-}

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -188,8 +188,8 @@ import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum.Type;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java

@@ -26,8 +26,8 @@ import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.inotify.MissingEventsException;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -68,7 +68,7 @@ import org.apache.hadoop.util.DataChecksum.Type;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.core.TraceScope;
+import org.apache.hadoop.tracing.TraceScope;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -938,8 +938,8 @@ public class DFSOutputStream extends FSOutputSummer
   void completeFile() throws IOException {
     // get last block before destroying the streamer
     ExtendedBlock lastBlock = getStreamer().getBlock();
-    try (TraceScope ignored =
-        dfsClient.getTracer().newScope("completeFile")) {
+    try (TraceScope ignored = dfsClient.getTracer()
+        .newScope("DFSOutputStream#completeFile")) {
       completeFile(lastBlock);
     }
   }
@@ -995,7 +995,10 @@ public class DFSOutputStream extends FSOutputSummer
           DFSClient.LOG.info(msg);
           throw new IOException(msg);
         }
-        try {
+        try (TraceScope scope = dfsClient.getTracer()
+            .newScope("DFSOutputStream#completeFile: Retry")) {
+          scope.addKVAnnotation("retries left", retries);
+          scope.addKVAnnotation("sleeptime (sleeping for)", sleeptime);
           if (retries == 0) {
             throw new IOException("Unable to close file because the last block "
                 + last + " does not have enough number of replicas.");

+ 16 - 17
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java

@@ -28,9 +28,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.TraceScope;
+import org.apache.hadoop.tracing.Span;
+import org.apache.hadoop.tracing.SpanContext;
 
 /****************************************************************
  * DFSPacket is used by DataStreamer and DFSOutputStream.
@@ -41,7 +40,7 @@ import org.apache.htrace.core.TraceScope;
 @InterfaceAudience.Private
 public class DFSPacket {
   public static final long HEART_BEAT_SEQNO = -1L;
-  private static SpanId[] EMPTY = new SpanId[0];
+  private static final SpanContext[] EMPTY = new SpanContext[0];
   private final long seqno; // sequence number of buffer in block
   private final long offsetInBlock; // offset in block
   private boolean syncBlock; // this packet forces the current block to disk
@@ -68,9 +67,9 @@ public class DFSPacket {
   private int checksumPos;
   private final int dataStart;
   private int dataPos;
-  private SpanId[] traceParents = EMPTY;
+  private SpanContext[] traceParents = EMPTY;
   private int traceParentsUsed;
-  private TraceScope scope;
+  private Span span;
 
   /**
    * Create a new packet.
@@ -306,11 +305,11 @@ public class DFSPacket {
     if (span == null) {
       return;
     }
-    addTraceParent(span.getSpanId());
+    addTraceParent(span.getContext());
   }
 
-  public void addTraceParent(SpanId id) {
-    if (!id.isValid()) {
+  public void addTraceParent(SpanContext ctx) {
+    if (ctx == null) {
       return;
     }
     if (traceParentsUsed == traceParents.length) {
@@ -318,7 +317,7 @@ public class DFSPacket {
           traceParents.length * 2;
       traceParents = Arrays.copyOf(traceParents, newLength);
     }
-    traceParents[traceParentsUsed] = id;
+    traceParents[traceParentsUsed] = ctx;
     traceParentsUsed++;
   }
 
@@ -329,17 +328,17 @@ public class DFSPacket {
    * <p>
    * Protected by the DFSOutputStream dataQueue lock.
    */
-  public SpanId[] getTraceParents() {
+  public SpanContext[] getTraceParents() {
     // Remove duplicates from the array.
     int len = traceParentsUsed;
     Arrays.sort(traceParents, 0, len);
     int i = 0, j = 0;
-    SpanId prevVal = SpanId.INVALID;
+    SpanContext prevVal = null;
     while (true) {
       if (i == len) {
         break;
       }
-      SpanId val = traceParents[i];
+      SpanContext val = traceParents[i];
       if (!val.equals(prevVal)) {
         traceParents[j] = val;
         j++;
@@ -354,11 +353,11 @@ public class DFSPacket {
     return traceParents;
   }
 
-  public void setTraceScope(TraceScope scope) {
-    this.scope = scope;
+  public void setSpan(Span span) {
+    this.span = span;
   }
 
-  public TraceScope getTraceScope() {
-    return scope;
+  public Span getSpan() {
+    return span;
   }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java

@@ -45,7 +45,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.core.TraceScope;
+import org.apache.hadoop.tracing.TraceScope;
 
 import java.io.IOException;
 import java.io.InterruptedIOException;

+ 25 - 23
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java

@@ -75,10 +75,10 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Span;
+import org.apache.hadoop.tracing.SpanContext;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 
 import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder;
 import org.apache.hadoop.thirdparty.com.google.common.cache.CacheLoader;
@@ -698,11 +698,14 @@ class DataStreamer extends Daemon {
               LOG.debug("Thread interrupted", e);
             }
             one = dataQueue.getFirst(); // regular data packet
-            SpanId[] parents = one.getTraceParents();
-            if (parents.length > 0) {
+            SpanContext[] parents = one.getTraceParents();
+            if (parents != null && parents.length > 0) {
+              // The original code stored multiple parents in the DFSPacket, and
+              // use them ALL here when creating a new Span. We only use the
+              // last one FOR NOW. Moreover, we don't activate the Span for now.
               scope = dfsClient.getTracer().
-                  newScope("dataStreamer", parents[0]);
-              scope.getSpan().setParents(parents);
+                  newScope("dataStreamer", parents[0], false);
+              //scope.getSpan().setParents(parents);
             }
           }
         }
@@ -748,14 +751,14 @@ class DataStreamer extends Daemon {
         }
 
         // send the packet
-        SpanId spanId = SpanId.INVALID;
+        SpanContext spanContext = null;
         synchronized (dataQueue) {
           // move packet from dataQueue to ackQueue
           if (!one.isHeartbeatPacket()) {
             if (scope != null) {
-              spanId = scope.getSpanId();
-              scope.detach();
-              one.setTraceScope(scope);
+              one.setSpan(scope.span());
+              spanContext = scope.span().getContext();
+              scope.close();
             }
             scope = null;
             dataQueue.removeFirst();
@@ -769,7 +772,7 @@ class DataStreamer extends Daemon {
 
         // write out data to remote datanode
         try (TraceScope ignored = dfsClient.getTracer().
-            newScope("DataStreamer#writeTo", spanId)) {
+            newScope("DataStreamer#writeTo", spanContext)) {
           one.writeTo(blockStream);
           blockStream.flush();
         } catch (IOException e) {
@@ -1171,10 +1174,10 @@ class DataStreamer extends Daemon {
           block.setNumBytes(one.getLastByteOffsetBlock());
 
           synchronized (dataQueue) {
-            scope = one.getTraceScope();
-            if (scope != null) {
-              scope.reattach();
-              one.setTraceScope(null);
+            if (one.getSpan() != null) {
+              scope = new TraceScope(new Span());
+              // TODO: Use scope = Tracer.curThreadTracer().activateSpan ?
+              one.setSpan(null);
             }
             lastAckedSeqno = seqno;
             pipelineRecoveryCount = 0;
@@ -1269,11 +1272,10 @@ class DataStreamer extends Daemon {
         synchronized (dataQueue) {
           DFSPacket endOfBlockPacket = dataQueue.remove();  // remove the end of block packet
           // Close any trace span associated with this Packet
-          TraceScope scope = endOfBlockPacket.getTraceScope();
-          if (scope != null) {
-            scope.reattach();
-            scope.close();
-            endOfBlockPacket.setTraceScope(null);
+          Span span = endOfBlockPacket.getSpan();
+          if (span != null) {
+            span.finish();
+            endOfBlockPacket.setSpan(null);
           }
           assert endOfBlockPacket.isLastPacketInBlock();
           assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
@@ -1949,7 +1951,7 @@ class DataStreamer extends Daemon {
   void queuePacket(DFSPacket packet) {
     synchronized (dataQueue) {
       if (packet == null) return;
-      packet.addTraceParent(Tracer.getCurrentSpanId());
+      packet.addTraceParent(Tracer.getCurrentSpan());
       dataQueue.addLast(packet);
       lastQueuedSeqno = packet.getSeqno();
       LOG.debug("Queued {}, {}", packet, this);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java

@@ -27,8 +27,8 @@ import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.ipc.RemoteException;
 
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 
 /**
  * CacheDirectiveIterator is a remote iterator that iterates cache directives.

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java

@@ -23,8 +23,8 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 
 /**
  * CachePoolIterator is a remote iterator that iterates cache pools.

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java

@@ -23,8 +23,8 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 
 /**
  * EncryptionZoneIterator is a remote iterator that iterates over encryption

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/OpenFilesIterator.java

@@ -24,8 +24,8 @@ import java.util.EnumSet;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 
 /**
  * OpenFilesIterator is a remote iterator that iterates over the open files list

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ReencryptionStatusIterator.java

@@ -20,8 +20,8 @@ package org.apache.hadoop.hdfs.protocol;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
 
 import java.io.IOException;
 

+ 9 - 15
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java

@@ -35,8 +35,9 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Span;
+import org.apache.hadoop.tracing.Tracer;
+import org.apache.hadoop.tracing.TraceUtils;
 
 /**
  * Static utilities for dealing with the protocol buffers used by the
@@ -87,23 +88,16 @@ public abstract class DataTransferProtoUtil {
     BaseHeaderProto.Builder builder =  BaseHeaderProto.newBuilder()
         .setBlock(PBHelperClient.convert(blk))
         .setToken(PBHelperClient.convert(blockToken));
-    SpanId spanId = Tracer.getCurrentSpanId();
-    if (spanId.isValid()) {
-      builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder()
-          .setTraceId(spanId.getHigh())
-          .setParentId(spanId.getLow()));
+    Span span = Tracer.getCurrentSpan();
+    if (span != null) {
+      DataTransferTraceInfoProto.Builder traceInfoProtoBuilder =
+          DataTransferTraceInfoProto.newBuilder().setSpanContext(
+              TraceUtils.spanContextToByteString(span.getContext()));
+      builder.setTraceInfo(traceInfoProtoBuilder);
     }
     return builder.build();
   }
 
-  public static SpanId fromProto(DataTransferTraceInfoProto proto) {
-    if ((proto != null) && proto.hasTraceId() &&
-          proto.hasParentId()) {
-      return new SpanId(proto.getTraceId(), proto.getParentId());
-    }
-    return null;
-  }
-
   public static void checkBlockOpStatus(
           BlockOpResponseProto response,
           String logInfo) throws IOException {

+ 15 - 12
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java

@@ -52,8 +52,9 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
 
-import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Span;
+import org.apache.hadoop.tracing.Tracer;
+import org.apache.hadoop.tracing.TraceUtils;
 
 import org.apache.hadoop.thirdparty.protobuf.Message;
 
@@ -212,11 +213,12 @@ public class Sender implements DataTransferProtocol {
     ReleaseShortCircuitAccessRequestProto.Builder builder =
         ReleaseShortCircuitAccessRequestProto.newBuilder().
             setSlotId(PBHelperClient.convert(slotId));
-    SpanId spanId = Tracer.getCurrentSpanId();
-    if (spanId.isValid()) {
-      builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
-          setTraceId(spanId.getHigh()).
-          setParentId(spanId.getLow()));
+    Span span = Tracer.getCurrentSpan();
+    if (span != null) {
+      DataTransferTraceInfoProto.Builder traceInfoProtoBuilder =
+          DataTransferTraceInfoProto.newBuilder().setSpanContext(
+              TraceUtils.spanContextToByteString(span.getContext()));
+      builder.setTraceInfo(traceInfoProtoBuilder);
     }
     ReleaseShortCircuitAccessRequestProto proto = builder.build();
     send(out, Op.RELEASE_SHORT_CIRCUIT_FDS, proto);
@@ -227,11 +229,12 @@ public class Sender implements DataTransferProtocol {
     ShortCircuitShmRequestProto.Builder builder =
         ShortCircuitShmRequestProto.newBuilder().
             setClientName(clientName);
-    SpanId spanId = Tracer.getCurrentSpanId();
-    if (spanId.isValid()) {
-      builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
-          setTraceId(spanId.getHigh()).
-          setParentId(spanId.getLow()));
+    Span span = Tracer.getCurrentSpan();
+    if (span != null) {
+      DataTransferTraceInfoProto.Builder traceInfoProtoBuilder =
+          DataTransferTraceInfoProto.newBuilder().setSpanContext(
+              TraceUtils.spanContextToByteString(span.getContext()));
+      builder.setTraceInfo(traceInfoProtoBuilder);
     }
     ShortCircuitShmRequestProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_SHM, proto);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/datatransfer.proto

@@ -58,8 +58,9 @@ message BaseHeaderProto {
 }
 
 message DataTransferTraceInfoProto {
-  required uint64 traceId = 1;
-  required uint64 parentId = 2;
+  optional uint64 traceId = 1;
+  optional uint64 parentId = 2;
+  optional bytes spanContext = 3;
 }
 
 message ClientOperationHeaderProto {

+ 0 - 26
hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/TestDFSPacket.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs;
 import java.util.Random;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
 import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.htrace.core.SpanId;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -66,29 +65,4 @@ public class TestDFSPacket {
       }
     }
   }
-
-  @Test
-  public void testAddParentsGetParents() throws Exception {
-    DFSPacket p = new DFSPacket(null, maxChunksPerPacket,
-                                0, 0, checksumSize, false);
-    SpanId parents[] = p.getTraceParents();
-    Assert.assertEquals(0, parents.length);
-    p.addTraceParent(new SpanId(0, 123));
-    p.addTraceParent(new SpanId(0, 123));
-    parents = p.getTraceParents();
-    Assert.assertEquals(1, parents.length);
-    Assert.assertEquals(new SpanId(0, 123), parents[0]);
-    parents = p.getTraceParents(); // test calling 'get' again.
-    Assert.assertEquals(1, parents.length);
-    Assert.assertEquals(new SpanId(0, 123), parents[0]);
-    p.addTraceParent(new SpanId(0, 1));
-    p.addTraceParent(new SpanId(0, 456));
-    p.addTraceParent(new SpanId(0, 789));
-    parents = p.getTraceParents();
-    Assert.assertEquals(4, parents.length);
-    Assert.assertEquals(new SpanId(0, 1), parents[0]);
-    Assert.assertEquals(new SpanId(0, 123), parents[1]);
-    Assert.assertEquals(new SpanId(0, 456), parents[2]);
-    Assert.assertEquals(new SpanId(0, 789), parents[3]);
-  }
 }

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs/pom.xml

@@ -180,10 +180,6 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
       <artifactId>netty-all</artifactId>
       <scope>compile</scope>
     </dependency>
-    <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core4</artifactId>
-    </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-kms</artifactId>

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HDFSPolicyProvider.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.security.authorize.Service;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
 import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
 import org.apache.hadoop.ipc.GenericRefreshProtocol;
-import org.apache.hadoop.tracing.TraceAdminProtocol;
 
 /**
  * {@link PolicyProvider} for HDFS protocols.
@@ -80,9 +79,6 @@ public class HDFSPolicyProvider extends PolicyProvider {
     new Service(
         CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_GENERIC_REFRESH,
         GenericRefreshProtocol.class),
-    new Service(
-        CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_TRACING,
-        TraceAdminProtocol.class),
     new Service(
         CommonConfigurationKeys.HADOOP_SECURITY_SERVICE_AUTHORIZATION_DATANODE_LIFELINE,
         DatanodeLifelineProtocol.class),

+ 16 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java

@@ -46,9 +46,11 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmR
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
-import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.SpanContext;
+import org.apache.hadoop.tracing.TraceScope;
+import org.apache.hadoop.tracing.Tracer;
+import org.apache.hadoop.tracing.TraceUtils;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
 
 /** Receiver */
 @InterfaceAudience.Private
@@ -77,12 +79,13 @@ public abstract class Receiver implements DataTransferProtocol {
     return Op.read(in);
   }
 
-  private TraceScope continueTraceSpan(DataTransferTraceInfoProto proto,
+  private TraceScope continueTraceSpan(ByteString spanContextBytes,
                                        String description) {
     TraceScope scope = null;
-    SpanId spanId = fromProto(proto);
-    if (spanId != null) {
-      scope = tracer.newScope(description, spanId);
+    SpanContext spanContext =
+        TraceUtils.byteStringToSpanContext(spanContextBytes);
+    if (spanContext != null) {
+      scope = tracer.newScope(description, spanContext);
     }
     return scope;
   }
@@ -94,7 +97,8 @@ public abstract class Receiver implements DataTransferProtocol {
 
   private TraceScope continueTraceSpan(BaseHeaderProto header,
                                              String description) {
-    return continueTraceSpan(header.getTraceInfo(), description);
+    return continueTraceSpan(header.getTraceInfo().getSpanContext(),
+        description);
   }
 
   /** Process op by the corresponding method. */
@@ -243,7 +247,8 @@ public abstract class Receiver implements DataTransferProtocol {
       throws IOException {
     final ReleaseShortCircuitAccessRequestProto proto =
       ReleaseShortCircuitAccessRequestProto.parseFrom(vintPrefixed(in));
-    TraceScope traceScope = continueTraceSpan(proto.getTraceInfo(),
+    TraceScope traceScope = continueTraceSpan(
+        proto.getTraceInfo().getSpanContext(),
         proto.getClass().getSimpleName());
     try {
       releaseShortCircuitFds(PBHelperClient.convert(proto.getSlotId()));
@@ -256,7 +261,8 @@ public abstract class Receiver implements DataTransferProtocol {
   private void opRequestShortCircuitShm(DataInputStream in) throws IOException {
     final ShortCircuitShmRequestProto proto =
         ShortCircuitShmRequestProto.parseFrom(vintPrefixed(in));
-    TraceScope traceScope = continueTraceSpan(proto.getTraceInfo(),
+    TraceScope traceScope = continueTraceSpan(
+        proto.getTraceInfo().getSpanContext(),
         proto.getClass().getSimpleName());
     try {
       requestShortCircuitShm(proto.getClientName());

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java

@@ -48,7 +48,7 @@ import static org.apache.hadoop.util.ExitUtil.terminate;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Tracer;
 import org.eclipse.jetty.util.ajax.JSON;
 
 import javax.management.ObjectName;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -58,8 +58,8 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Span;
+import org.apache.hadoop.tracing.Tracer;
 
 import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED;
 import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.SYNC_FILE_RANGE_WRITE;

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -34,6 +34,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -50,7 +51,7 @@ import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
 import org.apache.hadoop.net.SocketOutputStream;
 import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.core.TraceScope;
+import org.apache.hadoop.tracing.TraceScope;
 
 import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_DONTNEED;
 import static org.apache.hadoop.io.nativeio.NativeIO.POSIX.POSIX_FADV_SEQUENTIAL;
@@ -750,8 +751,8 @@ class BlockSender implements java.io.Closeable {
    */
   long sendBlock(DataOutputStream out, OutputStream baseStream, 
                  DataTransferThrottler throttler) throws IOException {
-    final TraceScope scope = datanode.getTracer().
-        newScope("sendBlock_" + block.getBlockId());
+    final TraceScope scope = FsTracer.get(null)
+        .newScope("sendBlock_" + block.getBlockId());
     try {
       return doSendBlock(out, baseStream, throttler);
     } finally {

+ 3 - 42
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -204,16 +204,10 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.tracing.SpanReceiverInfo;
-import org.apache.hadoop.tracing.TraceAdminPB.TraceAdminService;
-import org.apache.hadoop.tracing.TraceAdminProtocol;
-import org.apache.hadoop.tracing.TraceAdminProtocolPB;
-import org.apache.hadoop.tracing.TraceAdminProtocolServerSideTranslatorPB;
 import org.apache.hadoop.tracing.TraceUtils;
-import org.apache.hadoop.tracing.TracerConfigurationManager;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.concurrent.HadoopExecutors;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Tracer;
 import org.eclipse.jetty.util.ajax.JSON;
 
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -262,7 +256,7 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 public class DataNode extends ReconfigurableBase
     implements InterDatanodeProtocol, ClientDatanodeProtocol,
-        TraceAdminProtocol, DataNodeMXBean, ReconfigurationProtocol {
+        DataNodeMXBean, ReconfigurationProtocol {
   public static final Logger LOG = LoggerFactory.getLogger(DataNode.class);
   
   static{
@@ -389,7 +383,6 @@ public class DataNode extends ReconfigurableBase
   private BlockRecoveryWorker blockRecoveryWorker;
   private ErasureCodingWorker ecWorker;
   private final Tracer tracer;
-  private final TracerConfigurationManager tracerConfigurationManager;
   private static final int NUM_CORES = Runtime.getRuntime()
       .availableProcessors();
   private static final double CONGESTION_RATIO = 1.5;
@@ -406,7 +399,7 @@ public class DataNode extends ReconfigurableBase
 
   private static Tracer createTracer(Configuration conf) {
     return new Tracer.Builder("DataNode").
-        conf(TraceUtils.wrapHadoopConf(DATANODE_HTRACE_PREFIX , conf)).
+        conf(TraceUtils.wrapHadoopConf(DATANODE_HTRACE_PREFIX, conf)).
         build();
   }
 
@@ -422,8 +415,6 @@ public class DataNode extends ReconfigurableBase
   DataNode(final Configuration conf) throws DiskErrorException {
     super(conf);
     this.tracer = createTracer(conf);
-    this.tracerConfigurationManager =
-        new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf);
     this.fileIoProvider = new FileIoProvider(conf, this);
     this.fileDescriptorPassingDisabledReason = null;
     this.maxNumberOfBlocksToLog = 0;
@@ -451,8 +442,6 @@ public class DataNode extends ReconfigurableBase
            final SecureResources resources) throws IOException {
     super(conf);
     this.tracer = createTracer(conf);
-    this.tracerConfigurationManager =
-        new TracerConfigurationManager(DATANODE_HTRACE_PREFIX, conf);
     this.fileIoProvider = new FileIoProvider(conf, this);
     this.blockScanner = new BlockScanner(this);
     this.lastDiskErrorCheck = 0;
@@ -1038,16 +1027,6 @@ public class DataNode extends ReconfigurableBase
     DFSUtil.addPBProtocol(getConf(), InterDatanodeProtocolPB.class, service,
         ipcServer);
 
-    TraceAdminProtocolServerSideTranslatorPB traceAdminXlator =
-        new TraceAdminProtocolServerSideTranslatorPB(this);
-    BlockingService traceAdminService = TraceAdminService
-        .newReflectiveBlockingService(traceAdminXlator);
-    DFSUtil.addPBProtocol(
-        getConf(),
-        TraceAdminProtocolPB.class,
-        traceAdminService,
-        ipcServer);
-
     LOG.info("Opened IPC server at {}", ipcServer.getListenerAddress());
 
     // set service-level authorization security policy
@@ -3572,24 +3551,6 @@ public class DataNode extends ReconfigurableBase
     return lastDiskErrorCheck;
   }
 
-  @Override
-  public SpanReceiverInfo[] listSpanReceivers() throws IOException {
-    checkSuperuserPrivilege();
-    return tracerConfigurationManager.listSpanReceivers();
-  }
-
-  @Override
-  public long addSpanReceiver(SpanReceiverInfo info) throws IOException {
-    checkSuperuserPrivilege();
-    return tracerConfigurationManager.addSpanReceiver(info);
-  }
-
-  @Override
-  public void removeSpanReceiver(long id) throws IOException {
-    checkSuperuserPrivilege();
-    tracerConfigurationManager.removeSpanReceiver(id);
-  }
-
   public BlockRecoveryWorker getBlockRecoveryWorker(){
     return blockRecoveryWorker;
   }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -22,6 +22,7 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.thirdparty.protobuf.ByteString;
 import javax.crypto.SecretKey;
 import org.apache.commons.logging.Log;
+import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.ExtendedBlockId;
@@ -135,7 +136,7 @@ class DataXceiver extends Receiver implements Runnable {
   
   private DataXceiver(Peer peer, DataNode datanode,
       DataXceiverServer dataXceiverServer) throws IOException {
-    super(datanode.getTracer());
+    super(FsTracer.get(null));
     this.peer = peer;
     this.dnConf = datanode.getDnConf();
     this.socketIn = peer.getInputStream();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetUtil.java

@@ -34,6 +34,7 @@ import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.util.Arrays;
 
+import com.fasterxml.jackson.databind.util.ByteBufferBackedInputStream;
 import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
@@ -45,7 +46,6 @@ import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.htrace.shaded.fasterxml.jackson.databind.util.ByteBufferBackedInputStream;
 
 /** Utility methods. */
 @InterfaceAudience.Private

+ 1 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -88,9 +88,7 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.RefreshAuthorizationPolicyProtocol;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
-import org.apache.hadoop.tracing.TraceAdminProtocol;
 import org.apache.hadoop.tracing.TraceUtils;
-import org.apache.hadoop.tracing.TracerConfigurationManager;
 import org.apache.hadoop.util.ExitUtil.ExitException;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.JvmPauseMonitor;
@@ -99,7 +97,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.GcTimeMonitor;
 import org.apache.hadoop.util.GcTimeMonitor.Builder;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Tracer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -372,8 +370,6 @@ public class NameNode extends ReconfigurableBase implements
       return RefreshCallQueueProtocol.versionID;
     } else if (protocol.equals(GetUserMappingsProtocol.class.getName())){
       return GetUserMappingsProtocol.versionID;
-    } else if (protocol.equals(TraceAdminProtocol.class.getName())){
-      return TraceAdminProtocol.versionID;
     } else {
       throw new IOException("Unknown protocol to name node: " + protocol);
     }
@@ -428,7 +424,6 @@ public class NameNode extends ReconfigurableBase implements
   private GcTimeMonitor gcTimeMonitor;
   private ObjectName nameNodeStatusBeanName;
   protected final Tracer tracer;
-  protected final TracerConfigurationManager tracerConfigurationManager;
   ScheduledThreadPoolExecutor metricsLoggerTimer;
 
   /**
@@ -997,8 +992,6 @@ public class NameNode extends ReconfigurableBase implements
     this.tracer = new Tracer.Builder("NameNode").
         conf(TraceUtils.wrapHadoopConf(NAMENODE_HTRACE_PREFIX, conf)).
         build();
-    this.tracerConfigurationManager =
-        new TracerConfigurationManager(NAMENODE_HTRACE_PREFIX, conf);
     this.role = role;
     String nsId = getNameServiceId(conf);
     String namenodeId = HAUtil.getNameNodeId(conf, nsId);

+ 0 - 34
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -219,10 +219,6 @@ import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.tools.proto.GetUserMappingsProtocolProtos.GetUserMappingsProtocolService;
 import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolPB;
 import org.apache.hadoop.tools.protocolPB.GetUserMappingsProtocolServerSideTranslatorPB;
-import org.apache.hadoop.tracing.SpanReceiverInfo;
-import org.apache.hadoop.tracing.TraceAdminPB.TraceAdminService;
-import org.apache.hadoop.tracing.TraceAdminProtocolPB;
-import org.apache.hadoop.tracing.TraceAdminProtocolServerSideTranslatorPB;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionUtil;
 import org.slf4j.Logger;
@@ -342,11 +338,6 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     BlockingService reconfigurationPbService = ReconfigurationProtocolService
         .newReflectiveBlockingService(reconfigurationProtocolXlator);
 
-    TraceAdminProtocolServerSideTranslatorPB traceAdminXlator =
-        new TraceAdminProtocolServerSideTranslatorPB(this);
-    BlockingService traceAdminService = TraceAdminService
-        .newReflectiveBlockingService(traceAdminXlator);
-
     InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
     if (serviceRpcAddr != null) {
       String bindHost = nn.getServiceRpcServerBindHost(conf);
@@ -390,8 +381,6 @@ public class NameNodeRpcServer implements NamenodeProtocols {
           genericRefreshService, serviceRpcServer);
       DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class, 
           getUserMappingService, serviceRpcServer);
-      DFSUtil.addPBProtocol(conf, TraceAdminProtocolPB.class,
-          traceAdminService, serviceRpcServer);
 
       // Update the address with the correct port
       InetSocketAddress listenAddr = serviceRpcServer.getListenerAddress();
@@ -494,8 +483,6 @@ public class NameNodeRpcServer implements NamenodeProtocols {
         genericRefreshService, clientRpcServer);
     DFSUtil.addPBProtocol(conf, GetUserMappingsProtocolPB.class, 
         getUserMappingService, clientRpcServer);
-    DFSUtil.addPBProtocol(conf, TraceAdminProtocolPB.class,
-        traceAdminService, clientRpcServer);
 
     // set service-level authorization security policy
     if (serviceAuthEnabled =
@@ -2474,27 +2461,6 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     return new EventBatchList(batches, firstSeenTxid, maxSeenTxid, syncTxid);
   }
 
-  @Override // TraceAdminProtocol
-  public SpanReceiverInfo[] listSpanReceivers() throws IOException {
-    checkNNStartup();
-    namesystem.checkSuperuserPrivilege();
-    return nn.tracerConfigurationManager.listSpanReceivers();
-  }
-
-  @Override // TraceAdminProtocol
-  public long addSpanReceiver(SpanReceiverInfo info) throws IOException {
-    checkNNStartup();
-    namesystem.checkSuperuserPrivilege();
-    return nn.tracerConfigurationManager.addSpanReceiver(info);
-  }
-
-  @Override // TraceAdminProtocol
-  public void removeSpanReceiver(long id) throws IOException {
-    checkNNStartup();
-    namesystem.checkSuperuserPrivilege();
-    nn.tracerConfigurationManager.removeSpanReceiver(id);
-  }
-
   @Override // ClientProtocol
   public ErasureCodingPolicyInfo[] getErasureCodingPolicies()
       throws IOException {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -87,7 +87,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.tracing.TraceUtils;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.core.Tracer;
+import org.apache.hadoop.tracing.Tracer;
 
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
 

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocols.java

@@ -27,7 +27,6 @@ import org.apache.hadoop.security.RefreshUserMappingsProtocol;
 import org.apache.hadoop.ipc.RefreshCallQueueProtocol;
 import org.apache.hadoop.ipc.GenericRefreshProtocol;
 import org.apache.hadoop.tools.GetUserMappingsProtocol;
-import org.apache.hadoop.tracing.TraceAdminProtocol;
 
 /** The full set of RPC methods implemented by the Namenode.  */
 @InterfaceAudience.Private
@@ -42,6 +41,5 @@ public interface NamenodeProtocols
           RefreshCallQueueProtocol,
           GenericRefreshProtocol,
           GetUserMappingsProtocol,
-          HAServiceProtocol,
-          TraceAdminProtocol {
+          HAServiceProtocol {
 }

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java

@@ -57,7 +57,6 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.test.Whitebox;
-import org.apache.htrace.core.SpanId;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -296,7 +295,6 @@ public class TestDFSOutputStream {
         Whitebox.getInternalState(stream, "congestedNodes");
     congestedNodes.add(mock(DatanodeInfo.class));
     DFSPacket packet = mock(DFSPacket.class);
-    when(packet.getTraceParents()).thenReturn(new SpanId[] {});
     dataQueue.add(packet);
     stream.run();
     Assert.assertTrue(congestedNodes.isEmpty());

+ 0 - 180
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTraceAdmin.java

@@ -1,180 +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.tracing;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferTestCase;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.net.unix.TemporarySocketDirectory;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.htrace.core.Tracer;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.PrintStream;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayList;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-
-/**
- * Test cases for TraceAdmin.
- */
-public class TestTraceAdmin extends SaslDataTransferTestCase {
-  private static final String NEWLINE = System.getProperty("line.separator");
-  private final static int ONE_DATANODE = 1;
-
-  private String runTraceCommand(TraceAdmin trace, String... cmd)
-      throws Exception {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    PrintStream ps = new PrintStream(baos);
-    PrintStream oldStdout = System.out;
-    PrintStream oldStderr = System.err;
-    System.setOut(ps);
-    System.setErr(ps);
-    int ret = -1;
-    try {
-      ret = trace.run(cmd);
-    } finally {
-      try {
-        System.out.flush();
-      } finally {
-        System.setOut(oldStdout);
-        System.setErr(oldStderr);
-      }
-    }
-    return "ret:" + ret + ", " + baos.toString();
-  }
-
-  private String getHostPortForNN(MiniDFSCluster cluster) {
-    return "127.0.0.1:" + cluster.getNameNodePort();
-  }
-
-  private String getHostPortForDN(MiniDFSCluster cluster, int index) {
-    ArrayList<DataNode> dns = cluster.getDataNodes();
-    assertTrue(index >= 0 && index < dns.size());
-    return "127.0.0.1:" + dns.get(index).getIpcPort();
-  }
-
-  @Test
-  public void testNoOperator() throws Exception {
-    TraceAdmin trace = new TraceAdmin();
-    trace.setConf(new Configuration());
-    Assert.assertEquals("ret:1, You must specify an operation." + NEWLINE,
-        runTraceCommand(trace, "-host", "127.0.0.1:12346"));
-  }
-
-  @Test
-  public void testCreateAndDestroySpanReceiver() throws Exception {
-    Configuration conf = new Configuration();
-    conf = new Configuration();
-    conf.set(TraceUtils.DEFAULT_HADOOP_TRACE_PREFIX +
-        Tracer.SPAN_RECEIVER_CLASSES_KEY, "");
-    MiniDFSCluster cluster =
-        new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
-    cluster.waitActive();
-    TemporarySocketDirectory tempDir = new TemporarySocketDirectory();
-    String tracePath =
-        new File(tempDir.getDir(), "tracefile").getAbsolutePath();
-    try {
-      TraceAdmin trace = new TraceAdmin();
-      trace.setConf(conf);
-      Assert.assertEquals("ret:0, [no span receivers found]" + NEWLINE,
-          runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster)));
-      Assert.assertEquals("ret:0, Added trace span receiver 1 with " +
-          "configuration hadoop.htrace.local.file.span.receiver.path = " + tracePath + NEWLINE,
-          runTraceCommand(trace, "-add", "-host", getHostPortForNN(cluster),
-              "-class", "org.apache.htrace.core.LocalFileSpanReceiver",
-              "-Chadoop.htrace.local.file.span.receiver.path=" + tracePath));
-      String list =
-          runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster));
-      Assert.assertTrue(list.startsWith("ret:0"));
-      Assert.assertTrue(list.contains("1   org.apache.htrace.core.LocalFileSpanReceiver"));
-      Assert.assertEquals("ret:0, Removed trace span receiver 1" + NEWLINE,
-          runTraceCommand(trace, "-remove", "1", "-host",
-              getHostPortForNN(cluster)));
-      Assert.assertEquals("ret:0, [no span receivers found]" + NEWLINE,
-          runTraceCommand(trace, "-list", "-host", getHostPortForNN(cluster)));
-      Assert.assertEquals("ret:0, Added trace span receiver 2 with " +
-          "configuration hadoop.htrace.local.file.span.receiver.path = " + tracePath + NEWLINE,
-          runTraceCommand(trace, "-add", "-host", getHostPortForNN(cluster),
-              "-class", "LocalFileSpanReceiver",
-              "-Chadoop.htrace.local.file.span.receiver.path=" + tracePath));
-      Assert.assertEquals("ret:0, Removed trace span receiver 2" + NEWLINE,
-          runTraceCommand(trace, "-remove", "2", "-host",
-              getHostPortForNN(cluster)));
-    } finally {
-      cluster.shutdown();
-      tempDir.close();
-    }
-  }
-
-  /**
-   * Test running hadoop trace commands with -principal option against
-   * Kerberized NN and DN.
-   *
-   * @throws Exception
-   */
-  @Test
-  public void testKerberizedTraceAdmin() throws Exception {
-    MiniDFSCluster cluster = null;
-    final HdfsConfiguration conf = createSecureConfig(
-        "authentication,privacy");
-    try {
-      cluster = new MiniDFSCluster.Builder(conf)
-          .numDataNodes(ONE_DATANODE)
-          .build();
-      cluster.waitActive();
-      final String nnHost = getHostPortForNN(cluster);
-      final String dnHost = getHostPortForDN(cluster, 0);
-      // login using keytab and run commands
-      UserGroupInformation
-          .loginUserFromKeytabAndReturnUGI(getHdfsPrincipal(), getHdfsKeytab())
-          .doAs(new PrivilegedExceptionAction<Void>() {
-            @Override
-            public Void run() throws Exception {
-              // send trace command to NN
-              TraceAdmin trace = new TraceAdmin();
-              trace.setConf(conf);
-              final String[] nnTraceCmd = new String[] {
-                  "-list", "-host", nnHost, "-principal",
-                  conf.get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)};
-              int ret = trace.run(nnTraceCmd);
-              assertEquals(0, ret);
-              // send trace command to DN
-              final String[] dnTraceCmd = new String[] {
-                  "-list", "-host", dnHost, "-principal",
-                  conf.get(DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY)};
-              ret = trace.run(dnTraceCmd);
-              assertEquals(0, ret);
-              return null;
-            }
-          });
-    } finally {
-      if (cluster != null) {
-        cluster.shutdown();
-      }
-    }
-  }
-}

+ 0 - 223
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java

@@ -1,223 +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.tracing;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FsTracer;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestTracing {
-  private static MiniDFSCluster cluster;
-  private static DistributedFileSystem dfs;
-
-  private final static Configuration TRACING_CONF;
-  private final static Configuration NO_TRACING_CONF;
-
-  static {
-    NO_TRACING_CONF = new Configuration();
-    NO_TRACING_CONF.setLong("dfs.blocksize", 100 * 1024);
-
-    TRACING_CONF = new Configuration(NO_TRACING_CONF);
-    TRACING_CONF.set(CommonConfigurationKeys.FS_CLIENT_HTRACE_PREFIX +
-        Tracer.SPAN_RECEIVER_CLASSES_KEY,
-        SetSpanReceiver.class.getName());
-    TRACING_CONF.set(CommonConfigurationKeys.FS_CLIENT_HTRACE_PREFIX +
-        Tracer.SAMPLER_CLASSES_KEY, "AlwaysSampler");
-  }
-
-  @Test
-  public void testTracing() throws Exception {
-    // write and read without tracing started
-    String fileName = "testTracingDisabled.dat";
-    writeTestFile(fileName);
-    Assert.assertEquals(0, SetSpanReceiver.size());
-    readTestFile(fileName);
-    Assert.assertEquals(0, SetSpanReceiver.size());
-
-    writeTestFile("testReadTraceHooks.dat");
-
-    FsTracer.clear();
-    Tracer tracer = FsTracer.get(TRACING_CONF);
-    writeWithTracing(tracer);
-    readWithTracing(tracer);
-  }
-
-  private void writeWithTracing(Tracer tracer) throws Exception {
-    long startTime = System.currentTimeMillis();
-    TraceScope ts = tracer.newScope("testWriteTraceHooks");
-    writeTestFile("testWriteTraceHooks.dat");
-    long endTime = System.currentTimeMillis();
-    ts.close();
-
-    String[] expectedSpanNames = new String[]{
-        "testWriteTraceHooks",
-        "ClientProtocol#create",
-        "ClientNamenodeProtocol#create",
-        "ClientProtocol#fsync",
-        "ClientNamenodeProtocol#fsync",
-        "ClientProtocol#complete",
-        "ClientNamenodeProtocol#complete",
-        "newStreamForCreate",
-        "DFSOutputStream#write",
-        "DFSOutputStream#close",
-        "dataStreamer",
-        "OpWriteBlockProto",
-        "ClientProtocol#addBlock",
-        "ClientNamenodeProtocol#addBlock"
-    };
-    SetSpanReceiver.assertSpanNamesFound(expectedSpanNames);
-
-    // The trace should last about the same amount of time as the test
-    Map<String, List<Span>> map = SetSpanReceiver.getMap();
-    Span s = map.get("testWriteTraceHooks").get(0);
-    Assert.assertNotNull(s);
-
-    // Spans homed in the top trace shoud have same trace id.
-    // Spans having multiple parents (e.g. "dataStreamer" added by HDFS-7054)
-    // and children of them are exception.
-    String[] spansInTopTrace = new String[]{
-        "testWriteTraceHooks",
-        "ClientProtocol#create",
-        "ClientNamenodeProtocol#create",
-        "ClientProtocol#fsync",
-        "ClientNamenodeProtocol#fsync",
-        "ClientProtocol#complete",
-        "ClientNamenodeProtocol#complete",
-        "newStreamForCreate",
-        "DFSOutputStream#write",
-        "DFSOutputStream#close",
-    };
-    for (String desc : spansInTopTrace) {
-      for (Span span : map.get(desc)) {
-        Assert.assertEquals(ts.getSpan().getSpanId().getHigh(),
-                            span.getSpanId().getHigh());
-      }
-    }
-
-    // test for timeline annotation added by HADOOP-11242
-    Assert.assertEquals("called",
-        map.get("ClientProtocol#create")
-           .get(0).getTimelineAnnotations()
-           .get(0).getMessage());
-
-    SetSpanReceiver.clear();
-  }
-
-  private void readWithTracing(Tracer tracer) throws Exception {
-    long startTime = System.currentTimeMillis();
-    TraceScope ts = tracer.newScope("testReadTraceHooks");
-    readTestFile("testReadTraceHooks.dat");
-    ts.close();
-    long endTime = System.currentTimeMillis();
-
-    String[] expectedSpanNames = new String[]{
-        "testReadTraceHooks",
-        "ClientProtocol#getBlockLocations",
-        "ClientNamenodeProtocol#getBlockLocations",
-        "OpReadBlockProto"
-    };
-    SetSpanReceiver.assertSpanNamesFound(expectedSpanNames);
-
-    // The trace should last about the same amount of time as the test
-    Map<String, List<Span>> map = SetSpanReceiver.getMap();
-    Span s = map.get("testReadTraceHooks").get(0);
-    Assert.assertNotNull(s);
-
-    long spanStart = s.getStartTimeMillis();
-    long spanEnd = s.getStopTimeMillis();
-    Assert.assertTrue(spanStart - startTime < 100);
-    Assert.assertTrue(spanEnd - endTime < 100);
-
-    // There should only be one trace id as it should all be homed in the
-    // top trace.
-    for (Span span : SetSpanReceiver.getSpans()) {
-      System.out.println(span.toJson());
-    }
-    for (Span span : SetSpanReceiver.getSpans()) {
-      Assert.assertEquals(ts.getSpan().getSpanId().getHigh(),
-                          span.getSpanId().getHigh());
-    }
-    SetSpanReceiver.clear();
-  }
-
-  private void writeTestFile(String testFileName) throws Exception {
-    Path filePath = new Path(testFileName);
-    FSDataOutputStream stream = dfs.create(filePath);
-    for (int i = 0; i < 10; i++) {
-      byte[] data = RandomStringUtils.randomAlphabetic(102400).getBytes();
-      stream.write(data);
-    }
-    stream.hsync();
-    stream.close();
-  }
-
-  private void readTestFile(String testFileName) throws Exception {
-    Path filePath = new Path(testFileName);
-    FSDataInputStream istream = dfs.open(filePath, 10240);
-    ByteBuffer buf = ByteBuffer.allocate(10240);
-
-    int count = 0;
-    try {
-      while (istream.read(buf) > 0) {
-        count += 1;
-        buf.clear();
-        istream.seek(istream.getPos() + 5);
-      }
-    } catch (IOException ioe) {
-      // Ignore this it's probably a seek after eof.
-    } finally {
-      istream.close();
-    }
-  }
-
-  @Before
-  public void startCluster() throws IOException {
-    cluster = new MiniDFSCluster.Builder(NO_TRACING_CONF)
-        .numDataNodes(3)
-        .build();
-    cluster.waitActive();
-    dfs = cluster.getFileSystem();
-    SetSpanReceiver.clear();
-  }
-
-  @After
-  public void shutDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-    FsTracer.clear();
-  }
-}

+ 0 - 108
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java

@@ -1,108 +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.tracing;
-
-import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
-import static org.junit.Assume.assumeTrue;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FsTracer;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.net.unix.DomainSocket;
-import org.apache.hadoop.net.unix.TemporarySocketDirectory;
-import org.apache.hadoop.util.NativeCodeLoader;
-import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class TestTracingShortCircuitLocalRead {
-  private static Configuration conf;
-  private static MiniDFSCluster cluster;
-  private static DistributedFileSystem dfs;
-  private static TemporarySocketDirectory sockDir;
-  static final Path TEST_PATH = new Path("testShortCircuitTraceHooks");
-  static final int TEST_LENGTH = 1234;
-
-  @BeforeClass
-  public static void init() {
-    sockDir = new TemporarySocketDirectory();
-    DomainSocket.disableBindPathValidation();
-  }
-
-  @AfterClass
-  public static void shutdown() throws IOException {
-    sockDir.close();
-  }
-
-  @Test
-  public void testShortCircuitTraceHooks() throws IOException {
-    assumeTrue(NativeCodeLoader.isNativeCodeLoaded());
-    assumeNotWindows();
-    conf = new Configuration();
-    conf.set(TraceUtils.DEFAULT_HADOOP_TRACE_PREFIX +
-            Tracer.SPAN_RECEIVER_CLASSES_KEY,
-        SetSpanReceiver.class.getName());
-    conf.set(TraceUtils.DEFAULT_HADOOP_TRACE_PREFIX +
-            Tracer.SAMPLER_CLASSES_KEY,
-        "AlwaysSampler");
-    conf.setLong("dfs.blocksize", 100 * 1024);
-    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
-    conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY, false);
-    conf.set(DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
-        new File(sockDir.getDir(),
-            "testShortCircuitTraceHooks._PORT.sock").getAbsolutePath());
-    conf.set(DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY, "CRC32C");
-    cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(1)
-        .build();
-    dfs = cluster.getFileSystem();
-
-    try {
-      DFSTestUtil.createFile(dfs, TEST_PATH, TEST_LENGTH, (short)1, 5678L);
-
-      TraceScope ts = FsTracer.get(conf).
-          newScope("testShortCircuitTraceHooks");
-      FSDataInputStream stream = dfs.open(TEST_PATH);
-      byte buf[] = new byte[TEST_LENGTH];
-      IOUtils.readFully(stream, buf, 0, TEST_LENGTH);
-      stream.close();
-      ts.close();
-
-      String[] expectedSpanNames = {
-        "OpRequestShortCircuitAccessProto",
-        "ShortCircuitShmRequestProto"
-      };
-      SetSpanReceiver.assertSpanNamesFound(expectedSpanNames);
-    } finally {
-      dfs.close();
-      cluster.shutdown();
-    }
-  }
-}

+ 0 - 12
hadoop-project/pom.xml

@@ -138,8 +138,6 @@
     <jna.version>5.2.0</jna.version>
     <grizzly.version>2.2.21</grizzly.version>
     <gson.version>2.2.4</gson.version>
-    <htrace3.version>3.1.0-incubating</htrace3.version>
-    <htrace4.version>4.1.0-incubating</htrace4.version>
     <metrics.version>3.2.4</metrics.version>
     <netty3.version>3.10.6.Final</netty3.version>
     <netty4.version>4.1.50.Final</netty4.version>
@@ -1255,16 +1253,6 @@
         <artifactId>jsch</artifactId>
         <version>0.1.55</version>
       </dependency>
-      <dependency>
-        <groupId>org.apache.htrace</groupId>
-        <artifactId>htrace-core</artifactId>
-        <version>${htrace3.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.apache.htrace</groupId>
-        <artifactId>htrace-core4</artifactId>
-        <version>${htrace4.version}</version>
-      </dependency>
       <dependency>
         <groupId>org.jdom</groupId>
         <artifactId>jdom</artifactId>