You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@htrace.apache.org by cm...@apache.org on 2016/04/18 23:03:40 UTC

incubator-htrace git commit: HTRACE-355. TraceRunnable and TraceCallable should pull description from child thread instead of parent (Mike Drob via cmccabe)

Repository: incubator-htrace
Updated Branches:
  refs/heads/master fad64c469 -> 23e2dce78


HTRACE-355. TraceRunnable and TraceCallable should pull description from child thread instead of parent (Mike Drob via cmccabe)


Project: http://git-wip-us.apache.org/repos/asf/incubator-htrace/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-htrace/commit/23e2dce7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-htrace/tree/23e2dce7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-htrace/diff/23e2dce7

Branch: refs/heads/master
Commit: 23e2dce78ebbc1d1501399ff681c2bf7ddc36b71
Parents: fad64c4
Author: Colin P. Mccabe <cm...@apache.org>
Authored: Mon Apr 18 13:53:45 2016 -0700
Committer: Colin P. Mccabe <cm...@apache.org>
Committed: Mon Apr 18 13:54:17 2016 -0700

----------------------------------------------------------------------
 .../org/apache/htrace/core/TraceCallable.java   | 10 +--
 .../org/apache/htrace/core/TraceRunnable.java   | 10 +--
 .../java/org/apache/htrace/core/Tracer.java     |  4 +
 .../apache/htrace/core/TestTraceExecutor.java   | 86 ++++++++++++++++++++
 4 files changed, 100 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/23e2dce7/htrace-core4/src/main/java/org/apache/htrace/core/TraceCallable.java
----------------------------------------------------------------------
diff --git a/htrace-core4/src/main/java/org/apache/htrace/core/TraceCallable.java b/htrace-core4/src/main/java/org/apache/htrace/core/TraceCallable.java
index f5434e1..9cf478d 100644
--- a/htrace-core4/src/main/java/org/apache/htrace/core/TraceCallable.java
+++ b/htrace-core4/src/main/java/org/apache/htrace/core/TraceCallable.java
@@ -32,15 +32,15 @@ public class TraceCallable<V> implements Callable<V> {
     this.tracer = tracer;
     this.impl = impl;
     this.parent = parent;
-    if (description == null) {
-      this.description = Thread.currentThread().getName();
-    } else {
-      this.description = description;
-    }
+    this.description = description;
   }
 
   @Override
   public V call() throws Exception {
+    String description = this.description;
+    if (description == null) {
+      description = Thread.currentThread().getName();
+    }
     try (TraceScope chunk = tracer.newScope(description, parent.getSpan().getSpanId())) {
       return impl.call();
     }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/23e2dce7/htrace-core4/src/main/java/org/apache/htrace/core/TraceRunnable.java
----------------------------------------------------------------------
diff --git a/htrace-core4/src/main/java/org/apache/htrace/core/TraceRunnable.java b/htrace-core4/src/main/java/org/apache/htrace/core/TraceRunnable.java
index abf530f..f2db5c2 100644
--- a/htrace-core4/src/main/java/org/apache/htrace/core/TraceRunnable.java
+++ b/htrace-core4/src/main/java/org/apache/htrace/core/TraceRunnable.java
@@ -30,15 +30,15 @@ public class TraceRunnable implements Runnable {
     this.tracer = tracer;
     this.parent = parent;
     this.runnable = runnable;
-    if (description == null) {
-      this.description = Thread.currentThread().getName();
-    } else {
-      this.description = description;
-    }
+    this.description = description;
   }
 
   @Override
   public void run() {
+    String description = this.description;
+    if (description == null) {
+      description = Thread.currentThread().getName();
+    }
     try (TraceScope chunk = tracer.newScope(description, parent.getSpan().getSpanId())) {
       runnable.run();
     }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/23e2dce7/htrace-core4/src/main/java/org/apache/htrace/core/Tracer.java
----------------------------------------------------------------------
diff --git a/htrace-core4/src/main/java/org/apache/htrace/core/Tracer.java b/htrace-core4/src/main/java/org/apache/htrace/core/Tracer.java
index 9a5b04e..a04c9b9 100644
--- a/htrace-core4/src/main/java/org/apache/htrace/core/Tracer.java
+++ b/htrace-core4/src/main/java/org/apache/htrace/core/Tracer.java
@@ -465,6 +465,10 @@ public class Tracer implements Closeable {
     return new TraceRunnable(this, parentScope, runnable, description);
   }
 
+  public TraceExecutorService newTraceExecutorService(ExecutorService impl) {
+    return newTraceExecutorService(impl, null);
+  }
+
   public TraceExecutorService newTraceExecutorService(ExecutorService impl,
                                                       String scopeName) {
     return new TraceExecutorService(this, scopeName, impl);

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/23e2dce7/htrace-core4/src/test/java/org/apache/htrace/core/TestTraceExecutor.java
----------------------------------------------------------------------
diff --git a/htrace-core4/src/test/java/org/apache/htrace/core/TestTraceExecutor.java b/htrace-core4/src/test/java/org/apache/htrace/core/TestTraceExecutor.java
new file mode 100644
index 0000000..bf98a1a
--- /dev/null
+++ b/htrace-core4/src/test/java/org/apache/htrace/core/TestTraceExecutor.java
@@ -0,0 +1,86 @@
+/*
+ * 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.htrace.core;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.junit.Test;
+
+public class TestTraceExecutor {
+  private static final int WAIT_TIME_SECONDS = 60;
+
+  @Test
+  public void testExecutorWithNullScope() throws Exception {
+    HTraceConfiguration conf = HTraceConfiguration.fromKeyValuePairs("sampler.classes", "AlwaysSampler");
+    ExecutorService es = null;
+    try (Tracer tracer = new Tracer.Builder("TestTraceExecutor").conf(conf).build()) {
+      es = Executors.newSingleThreadExecutor(new NamingThreadFactory());
+      es = tracer.newTraceExecutorService(es);
+
+      try (TraceScope scope = tracer.newScope("TestRunnable")) {
+        final AtomicReference<String> description = new AtomicReference<String>("");
+        es.submit(new Runnable() {
+          @Override
+          public void run() {
+            description.set(Tracer.getCurrentSpan().getDescription());
+          }
+        }).get(WAIT_TIME_SECONDS, TimeUnit.SECONDS);
+        assertEquals("TraceRunnable did not set description correctly", "child-thread-1", description.get());
+      }
+
+      try (TraceScope scope = tracer.newScope("TestCallable")) {
+        String description = es.submit(new Callable<String>() {
+          @Override
+          public String call() throws Exception {
+            return Tracer.getCurrentSpan().getDescription();
+          }
+        }).get(WAIT_TIME_SECONDS, TimeUnit.SECONDS);
+        assertEquals("TraceCallable did not set description correctly", "child-thread-1", description);
+      }
+    } finally {
+      if (es != null) {
+        es.shutdown();
+      }
+    }
+  }
+
+  /*
+   * Inspired by org.apache.solr.util.DefaultSolrThreadFactory
+   */
+  static class NamingThreadFactory implements ThreadFactory {
+    private final ThreadGroup group;
+    private final AtomicInteger threadNumber = new AtomicInteger(1);
+
+    public NamingThreadFactory() {
+      SecurityManager s = System.getSecurityManager();
+      group = (s != null) ? s.getThreadGroup() : Thread.currentThread().getThreadGroup();
+    }
+
+    @Override
+    public Thread newThread(Runnable r) {
+      return new Thread(group, r, "child-thread-" + threadNumber.getAndIncrement(), 0);
+    }
+  }
+}