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 2015/08/25 04:18:08 UTC

[1/2] incubator-htrace git commit: HTRACE-214. De-globalize Tracer.java (cmccabe)

Repository: incubator-htrace
Updated Branches:
  refs/heads/master 0f873fdec -> 7997d2089


http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/TracerPool.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/TracerPool.java b/htrace-core/src/main/java/org/apache/htrace/core/TracerPool.java
new file mode 100644
index 0000000..ea4d145
--- /dev/null
+++ b/htrace-core/src/main/java/org/apache/htrace/core/TracerPool.java
@@ -0,0 +1,283 @@
+/*
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+
+/**
+ * A pool of Tracer objects.
+ */
+public class TracerPool {
+  private static final Log LOG = LogFactory.getLog(TracerPool.class);
+
+  /**
+   * The global pool of tracer objects.
+   *
+   * This is the pool that new tracers get put into by default.
+   */
+  static final TracerPool GLOBAL = new TracerPool("Global");
+
+  /**
+   * The shutdown hook which closes the Tracers in this pool when the process is
+   * shutting down.
+   */
+  private class SpanReceiverShutdownHook extends Thread {
+    SpanReceiverShutdownHook() {
+      setName("SpanReceiverShutdownHook");
+      setDaemon(false);
+    }
+
+    @Override
+    public void run() {
+      removeAndCloseAllSpanReceivers();
+    }
+  }
+
+  /**
+   * The name of this TracerPool.
+   */
+  private final String name;
+
+  /**
+   * The current span receivers which these tracers are using.
+   *
+   * Can be read locklessly.  Must be written under the lock.
+   * The array itself should never be modified.
+   */
+  private volatile SpanReceiver[] curReceivers;
+
+  /**
+   * The currently installed shutdown hook, or null if no hook has been
+   * installed.
+   */
+  private SpanReceiverShutdownHook shutdownHook;
+
+  /**
+   * The current Tracers.
+   */
+  private final HashSet<Tracer> curTracers;
+
+  /**
+   * Get the global tracer pool.
+   */
+  public static TracerPool getGlobalTracerPool() {
+    return GLOBAL;
+  }
+
+  public TracerPool(String name) {
+    this.name = name;
+    this.shutdownHook = null;
+    this.curTracers = new HashSet<Tracer>();
+    this.curReceivers = new SpanReceiver[0];
+  }
+
+  /**
+   * Return the name of this TracerPool.
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Returns an array of all the current span receivers.
+   *
+   * Note that if the current span receivers change, those changes will not be
+   * reflected in this array.  In other words, this array may be stale.
+   */
+  public SpanReceiver[] getReceivers() {
+    return curReceivers;
+  }
+
+  /**
+   * Add a new span receiver.
+   *
+   * @param receiver        The new receiver to add.
+   *
+   * @return                True if the new receiver was added; false if it
+   *                          already was there.
+   */
+  public synchronized boolean addReceiver(SpanReceiver receiver) {
+    SpanReceiver[] receivers = curReceivers;
+    int j = 0;
+    for (int i = 0; i < receivers.length; i++) {
+      if (receivers[i] == receiver) {
+        LOG.trace(toString() + ": can't add receiver " + receiver.toString() +
+                  " since it is already in this pool.");
+        return false;
+      }
+    }
+    SpanReceiver[] newReceivers =
+        Arrays.copyOf(receivers, receivers.length + 1);
+    newReceivers[receivers.length] = receiver;
+    registerShutdownHookIfNeeded();
+    curReceivers = newReceivers;
+    LOG.trace(toString() + ": added receiver " + receiver.toString());
+    return true;
+  }
+
+  /**
+   * Register the shutdown hook if needed.
+   */
+  private synchronized void registerShutdownHookIfNeeded() {
+    if (shutdownHook != null) {
+      return;
+    }
+    shutdownHook = new SpanReceiverShutdownHook();
+    Runtime.getRuntime().addShutdownHook(shutdownHook);
+    LOG.trace(toString() + ": registered shutdown hook.");
+  }
+
+  /**
+   * Remove a span receiver.
+   *
+   * @param receiver        The receiver to remove.
+   *
+   * @return                True if the receiver was removed; false if it
+   *                          did not exist in this pool.
+   */
+  public synchronized boolean removeReceiver(SpanReceiver receiver) {
+    SpanReceiver[] receivers = curReceivers;
+    int j = 0;
+    for (int i = 0; i < receivers.length; i++) {
+      if (receivers[i] == receiver) {
+        SpanReceiver[] newReceivers = new SpanReceiver[receivers.length - 1];
+        System.arraycopy(receivers, 0, newReceivers, 0, i);
+        System.arraycopy(receivers, i + 1, newReceivers, i,
+            receivers.length - i - 1);
+        curReceivers = newReceivers;
+        LOG.trace(toString() + ": removed receiver " + receiver.toString());
+        return true;
+      }
+    }
+    LOG.trace(toString() + ": can't remove receiver " + receiver.toString() +
+        " since it's not currently in this pool.");
+    return false;
+  }
+
+  /**
+   * Remove and close a span receiver.
+   *
+   * @param receiver        The receiver to remove.
+   *
+   * @return                True if the receiver was removed; false if it
+   *                          did not exist in this pool.
+   */
+  public boolean removeAndCloseReceiver(SpanReceiver receiver) {
+    if (!removeReceiver(receiver)) {
+      return false;
+    }
+    try {
+      LOG.trace(toString() + ": closing receiver " + receiver.toString());
+      receiver.close();
+    } catch (Throwable t) {
+      LOG.error(toString() + ": error closing " + receiver.toString(), t);
+    }
+    return true;
+  }
+
+  /**
+   * Remove and close all of the span receivers.
+   */
+  private synchronized void removeAndCloseAllSpanReceivers() {
+    SpanReceiver[] receivers = curReceivers;
+    curReceivers = new SpanReceiver[0];
+    for (SpanReceiver receiver : receivers) {
+      try {
+        LOG.trace(toString() + ": closing receiver " + receiver.toString());
+        receiver.close();
+      } catch (Throwable t) {
+        LOG.error(toString() + ": error closing " + receiver.toString(), t);
+      }
+    }
+  }
+
+  /**
+   * Given a SpanReceiver class name, return the existing instance of that span
+   * receiver, if possible; otherwise, invoke the callable to create a new
+   * instance.
+   *
+   * @param className       The span receiver class name.
+   * @param conf            The HTrace configuration.
+   * @param classLoader     The class loader to use.
+   *
+   * @return                The SpanReceiver.
+   */
+  public synchronized SpanReceiver loadReceiverType(String className,
+      HTraceConfiguration conf, ClassLoader classLoader) {
+    SpanReceiver[] receivers = curReceivers;
+    for (SpanReceiver receiver : receivers) {
+      if (receiver.getClass().getName().equals(className)) {
+        LOG.trace(toString() + ": returning a reference to receiver " +
+                  receiver.toString());
+        return receiver;
+      }
+    }
+    LOG.trace(toString() + ": creating a new SpanReceiver of type " +
+              className);
+    SpanReceiver receiver = new SpanReceiver.Builder(conf).
+        className(className).
+        classLoader(classLoader).
+        build();
+    addReceiver(receiver);
+    return receiver;
+  }
+
+  /**
+   * Returns an array of all the current Tracers.
+   *
+   * Note that if the current Tracers change, those changes will not be
+   * reflected in this array.  In other words, this array may be stale.
+   */
+  public synchronized Tracer[] getTracers() {
+    return curTracers.toArray(new Tracer[curTracers.size()]);
+  }
+
+  /**
+   * Add a new Tracer.
+   */
+  synchronized void addTracer(Tracer tracer) {
+    if (curTracers.add(tracer)) {
+      LOG.trace(toString() + ": adding tracer " + tracer.toString());
+    }
+  }
+
+  /**
+   * Remove a Tracer.
+   *
+   * If the Tracer removed was the last one, we will close all the SpanReceiver
+   * objects that we're managing.
+   */
+  synchronized void removeTracer(Tracer tracer) {
+    if (curTracers.remove(tracer)) {
+      LOG.trace(toString() + ": removing tracer " + tracer.toString());
+      if (curTracers.size() == 0) {
+        removeAndCloseAllSpanReceivers();
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "TracerPool(" + name + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/test/java/org/apache/htrace/core/TestBadClient.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/test/java/org/apache/htrace/core/TestBadClient.java b/htrace-core/src/test/java/org/apache/htrace/core/TestBadClient.java
index 54de21b..e30ea3e 100644
--- a/htrace-core/src/test/java/org/apache/htrace/core/TestBadClient.java
+++ b/htrace-core/src/test/java/org/apache/htrace/core/TestBadClient.java
@@ -16,6 +16,12 @@
  */
 package org.apache.htrace.core;
 
+import java.io.File;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.hamcrest.CoreMatchers.containsString;
@@ -24,57 +30,121 @@ import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
 
-import java.io.File;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-
 public class TestBadClient {
+  @After
+  public void clearBadState() {
+    // Clear the bad trace state so that we don't disrupt other unit tests
+    // that run in this JVM.
+    Tracer.threadLocalScope.set(null);
+  }
+
   /**
    * Test closing an outer scope when an inner one is still active.
    */
   @Test
   public void TestClosingOuterScope() throws Exception {
+    Tracer tracer = new TracerBuilder().
+        name("TestClosingOuterScopeTracer").
+        tracerPool(new TracerPool("TestClosingOuterScope")).
+        conf(HTraceConfiguration.
+            fromKeyValuePairs("sampler.classes", "AlwaysSampler")).build();
     boolean gotException = false;
-    TraceScope outerScope = Trace.startSpan("outer", AlwaysSampler.INSTANCE);
-    TraceScope innerScope = Trace.startSpan("inner");
+    TraceScope outerScope = tracer.newScope("outer");
+    TraceScope innerScope = tracer.newScope("inner");
     try {
       outerScope.close();
     } catch (RuntimeException e) {
       assertThat(e.getMessage(),
-          containsString("You have probably forgotten to close or detach"));
+          containsString("it is not the current TraceScope"));
       gotException = true;
     }
     assertTrue("Expected to get exception because of improper " +
         "scope closure.", gotException);
     innerScope.close();
+    tracer.close();
   }
 
   /**
    * Test calling detach() two times on a scope object.
    */
   @Test
-  public void TestDoubleDetach() throws Exception {
+  public void TestDoubleDetachIsCaught() throws Exception {
+    Tracer tracer = new TracerBuilder().
+        name("TestDoubleDetach").
+        tracerPool(new TracerPool("TestDoubleDetachIsCaught")).
+        conf(HTraceConfiguration.
+            fromKeyValuePairs("sampler.classes", "AlwaysSampler")).build();
     boolean gotException = false;
-    TraceScope myScope = Trace.startSpan("myScope", AlwaysSampler.INSTANCE);
+    TraceScope myScope = tracer.newScope("myScope");
     myScope.detach();
     try {
       myScope.detach();
     } catch (RuntimeException e) {
       assertThat(e.getMessage(),
-          containsString("it has already been detached."));
+          containsString("it is already detached."));
       gotException = true;
     }
     assertTrue("Expected to get exception because of double TraceScope " +
         "detach.", gotException);
+    tracer.close();
+  }
+
+  /**
+   * Test calling detach() two times on a scope object.
+   */
+  @Test
+  public void TestDoubleDetachOnNullScope() throws Exception {
+    Tracer tracer = new TracerBuilder().
+        name("TestDoubleDetachOnNullScope").
+        tracerPool(new TracerPool("TestDoubleDetachOnNullScope")).
+        conf(HTraceConfiguration.
+            fromKeyValuePairs("sampler.classes", "NeverSampler")).build();
+    boolean gotException = false;
+    TraceScope myScope = tracer.newScope("myScope");
+    myScope.detach();
+    try {
+      myScope.detach();
+    } catch (RuntimeException e) {
+      assertThat(e.getMessage(),
+          containsString("it is already detached."));
+      gotException = true;
+    }
+    assertTrue("Expected to get exception because of double TraceScope " +
+        "detach on NullScope.", gotException);
+    tracer.close();
+  }
+
+  /**
+   * Test calling reattach() two times on a scope object.
+   */
+  @Test
+  public void TestDoubleReattachIsCaught() throws Exception {
+    Tracer tracer = new TracerBuilder().
+        name("TestDoubleReattach").
+        tracerPool(new TracerPool("TestDoubleReattachIsCaught")).
+        conf(HTraceConfiguration.
+            fromKeyValuePairs("sampler.classes", "AlwaysSampler")).build();
+    boolean gotException = false;
+    TraceScope myScope = tracer.newScope("myScope");
+    myScope.detach();
+    myScope.reattach();
+    try {
+      myScope.reattach();
+    } catch (RuntimeException e) {
+      assertThat(e.getMessage(),
+          containsString("it is not detached."));
+      gotException = true;
+    }
+    assertTrue("Expected to get exception because of double TraceScope " +
+        "reattach.", gotException);
+    tracer.close();
   }
 
-  private static class SpanHolder {
-    Span span;
+  private static class ScopeHolder {
+    TraceScope scope;
 
-    void set(Span span) {
-      this.span = span;
+    void set(TraceScope scope) {
+      this.scope = scope;
     }
   }
 
@@ -83,64 +153,53 @@ public class TestBadClient {
    */
   @Test
   public void TestPassingSpanBetweenThreads() throws Exception {
-    final SpanHolder spanHolder = new SpanHolder();
+    final Tracer tracer = new TracerBuilder().
+        name("TestPassingSpanBetweenThreads").
+        tracerPool(new TracerPool("TestPassingSpanBetweenThreads")).
+        conf(HTraceConfiguration.
+            fromKeyValuePairs("sampler.classes", "AlwaysSampler")).build();
+    POJOSpanReceiver receiver =
+        new POJOSpanReceiver(HTraceConfiguration.EMPTY);
+    tracer.getTracerPool().addReceiver(receiver);
+    final ScopeHolder scopeHolder = new ScopeHolder();
     Thread th = new Thread(new Runnable() {
       @Override
       public void run() {
-        TraceScope workerScope = Trace.startSpan("workerSpan",
-            AlwaysSampler.INSTANCE);
-        spanHolder.set(workerScope.getSpan());
+        TraceScope workerScope = tracer.newScope("workerSpan");
         workerScope.detach();
+        scopeHolder.set(workerScope);
       }
     });
     th.start();
     th.join();
+    TraceScope workerScope = scopeHolder.scope;
+    SpanId workerScopeId = workerScope.getSpan().getSpanId();
 
-    // Create new scope whose parent is the worker thread's span. 
-    TraceScope outermost = Trace.startSpan("outermost", spanHolder.span);
-    TraceScope nested = Trace.startSpan("nested");
+    // Create new scope whose parent is the worker thread's span.
+    workerScope.reattach();
+    TraceScope nested = tracer.newScope("nested");
     nested.close();
-    outermost.close();
     // Create another span which also descends from the worker thread's span.
-    TraceScope nested2 = Trace.startSpan("nested2", spanHolder.span);
+    TraceScope nested2 = tracer.newScope("nested2");
     nested2.close();
 
     // Close the worker thread's span.
-    spanHolder.span.stop();
+    workerScope.close();
 
     // We can create another descendant, even though the worker thread's span
     // has been stopped.
-    TraceScope lateChildScope = Trace.startSpan("lateChild", spanHolder.span);
+    TraceScope lateChildScope = tracer.newScope("lateChild", workerScopeId);
     lateChildScope.close();
-  }
-
-  /**
-   * Test trying to manually set our TraceScope's parent in a case where there
-   * is a currently active span.
-   */
-  @Test
-  public void TestIncorrectStartSpan() throws Exception {
-    // Create new scope
-    TraceScope outermost = Trace.startSpan("outermost",
-        AlwaysSampler.INSTANCE);
-    // Create nested scope
-    TraceScope nested = Trace.startSpan("nested", outermost.getSpan()); 
-    // Error
-    boolean gotException = false;
-    try {
-      TraceScope error = Trace.startSpan("error", outermost.getSpan()); 
-      error.close();
-    } catch (RuntimeException e) {
-      assertThat(e.getMessage(),
-          containsString("there is already a currentSpan"));
-      gotException = true;
-    }
-    assertTrue("Expected to get exception because of incorrect startSpan.",
-        gotException);
-  }
+    tracer.close();
 
-  @After
-  public void resetCurrentSpan() {
-    Tracer.getInstance().setCurrentSpan(null);
+    TraceGraph traceGraph = new TraceGraph(receiver.getSpans());
+    Collection<Span> rootSpans =
+        traceGraph.getSpansByParent().find(SpanId.INVALID);
+    Assert.assertEquals(1, rootSpans.size());
+    Assert.assertEquals(workerScopeId,
+        rootSpans.iterator().next().getSpanId());
+    Collection<Span> childSpans =
+        traceGraph.getSpansByParent().find(workerScopeId);
+    Assert.assertEquals(3, childSpans.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/test/java/org/apache/htrace/core/TestHTrace.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/test/java/org/apache/htrace/core/TestHTrace.java b/htrace-core/src/test/java/org/apache/htrace/core/TestHTrace.java
index f1839cb..1771407 100644
--- a/htrace-core/src/test/java/org/apache/htrace/core/TestHTrace.java
+++ b/htrace-core/src/test/java/org/apache/htrace/core/TestHTrace.java
@@ -16,7 +16,6 @@
  */
 package org.apache.htrace.core;
 
-import java.io.File;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
@@ -24,93 +23,108 @@ import java.util.Map;
 import org.apache.htrace.core.TraceGraph.SpansByParent;
 
 import org.junit.Assert;
-import org.junit.Rule;
 import org.junit.Test;
 
 public class TestHTrace {
+  @Test
+  public void TestTracerCreateAndClose() throws Exception {
+    Tracer tracer = new TracerBuilder().
+        name("TestSimpleScope").
+        tracerPool(new TracerPool("TestTracerCreateAndClose")).
+        conf(HTraceConfiguration.fromKeyValuePairs(
+            "sampler.classes", "AlwaysSampler")).
+        build();
+    POJOSpanReceiver receiver =
+        new POJOSpanReceiver(HTraceConfiguration.EMPTY);
+    tracer.getTracerPool().addReceiver(receiver);
+    tracer.close();
+    Assert.assertTrue(receiver.getSpans().isEmpty());
+  }
 
-  @Rule
-  public TraceCreator traceCreator = new TraceCreator();
-
-  public static final String SPAN_FILE_FLAG = "spanFile";
-
-  /**
-   * Basic system test of HTrace.
-   *
-   * @throws Exception
-   */
   @Test
-  public void testHtrace() throws Exception {
-    final int numTraces = 3;
-    String fileName = System.getProperty(SPAN_FILE_FLAG);
+  public void TestSimpleScope() throws Exception {
+    Tracer tracer = new TracerBuilder().
+        name("TestSimpleScope").
+        tracerPool(new TracerPool("TestSimpleScope")).
+        conf(HTraceConfiguration.fromKeyValuePairs(
+            "sampler.classes", "AlwaysSampler")).
+        build();
+    POJOSpanReceiver receiver =
+        new POJOSpanReceiver(HTraceConfiguration.EMPTY);
+    tracer.getTracerPool().addReceiver(receiver);
+    TraceScope scope = tracer.newScope("Foo");
+    scope.close();
+    tracer.close();
+    Assert.assertEquals(1, receiver.getSpans().size());
+    Span span = receiver.getSpans().iterator().next();
+    Assert.assertEquals(0, span.getParents().length);
+  }
 
-    // writes spans to a file if one is provided to maven with
-    // -DspanFile="FILENAME", otherwise writes to standard out.
-    if (fileName != null) {
-      File f = new File(fileName);
-      File parent = f.getParentFile();
-      if (parent != null && !parent.exists() && !parent.mkdirs()) {
-        throw new IllegalArgumentException("Couldn't create file: "
-            + fileName);
-      }
-      HashMap<String, String> conf = new HashMap<String, String>();
-      conf.put("local-file-span-receiver.path", fileName);
-      LocalFileSpanReceiver receiver =
-          new LocalFileSpanReceiver(HTraceConfiguration.fromMap(conf));
-      traceCreator.addReceiver(receiver);
-    } else {
-      traceCreator.addReceiver(new StandardOutSpanReceiver(HTraceConfiguration.EMPTY));
+  @Test
+  public void TestCreateSpans() throws Exception {
+    Tracer tracer = new TracerBuilder().
+        name("TestCreateSpans").
+        tracerPool(new TracerPool("TestCreateSpans")).
+        conf(HTraceConfiguration.fromKeyValuePairs(
+            "sampler.classes", "AlwaysSampler")).
+        build();
+    POJOSpanReceiver receiver =
+        new POJOSpanReceiver(HTraceConfiguration.EMPTY);
+    tracer.getTracerPool().addReceiver(receiver);
+    TraceCreator traceCreator = new TraceCreator(tracer);
+    traceCreator.createSampleRpcTrace();
+    traceCreator.createSimpleTrace();
+    traceCreator.createThreadedTrace();
+    tracer.close();
+    TraceGraph traceGraph = new TraceGraph(receiver.getSpans());
+    Collection<Span> roots = traceGraph.getSpansByParent().find(SpanId.INVALID);
+    Assert.assertTrue("Trace tree must have roots", !roots.isEmpty());
+    Assert.assertEquals(3, roots.size());
+
+    Map<String, Span> descriptionToRootSpan = new HashMap<String, Span>();
+    for (Span root : roots) {
+      descriptionToRootSpan.put(root.getDescription(), root);
     }
 
-    traceCreator.addReceiver(new POJOSpanReceiver(HTraceConfiguration.EMPTY){
-      @Override
-      public void close() {
-        TraceGraph traceGraph = new TraceGraph(getSpans());
-        Collection<Span> roots = traceGraph.getSpansByParent().find(SpanId.INVALID);
-        Assert.assertTrue("Trace tree must have roots", !roots.isEmpty());
-        Assert.assertEquals(numTraces, roots.size());
-
-        Map<String, Span> descriptionToRootSpan = new HashMap<String, Span>();
-        for (Span root : roots) {
-          descriptionToRootSpan.put(root.getDescription(), root);
-        }
+    Assert.assertTrue(descriptionToRootSpan.keySet().contains(
+        TraceCreator.RPC_TRACE_ROOT));
+    Assert.assertTrue(descriptionToRootSpan.keySet().contains(
+        TraceCreator.SIMPLE_TRACE_ROOT));
+    Assert.assertTrue(descriptionToRootSpan.keySet().contains(
+        TraceCreator.THREADED_TRACE_ROOT));
 
-        Assert.assertTrue(descriptionToRootSpan.keySet().contains(
-            TraceCreator.RPC_TRACE_ROOT));
-        Assert.assertTrue(descriptionToRootSpan.keySet().contains(
-            TraceCreator.SIMPLE_TRACE_ROOT));
-        Assert.assertTrue(descriptionToRootSpan.keySet().contains(
-            TraceCreator.THREADED_TRACE_ROOT));
+    SpansByParent spansByParentId = traceGraph.getSpansByParent();
 
-        SpansByParent spansByParentId = traceGraph.getSpansByParent();
-        Span rpcTraceRoot = descriptionToRootSpan.get(TraceCreator.RPC_TRACE_ROOT);
-        Assert.assertEquals(1, spansByParentId.find(rpcTraceRoot.getSpanId()).size());
+    Span rpcTraceRoot = descriptionToRootSpan.get(TraceCreator.RPC_TRACE_ROOT);
+    Assert.assertEquals(1, spansByParentId.find(rpcTraceRoot.getSpanId()).size());
 
-        Span rpcTraceChild1 = spansByParentId.find(rpcTraceRoot.getSpanId())
-            .iterator().next();
-        Assert.assertEquals(1, spansByParentId.find(rpcTraceChild1.getSpanId()).size());
+    Span rpcTraceChild1 = spansByParentId.find(rpcTraceRoot.getSpanId())
+        .iterator().next();
+    Assert.assertEquals(1, spansByParentId.find(rpcTraceChild1.getSpanId()).size());
 
-        Span rpcTraceChild2 = spansByParentId.find(rpcTraceChild1.getSpanId())
-            .iterator().next();
-        Assert.assertEquals(1, spansByParentId.find(rpcTraceChild2.getSpanId()).size());
+    Span rpcTraceChild2 = spansByParentId.find(rpcTraceChild1.getSpanId())
+        .iterator().next();
+    Assert.assertEquals(1, spansByParentId.find(rpcTraceChild2.getSpanId()).size());
 
-        Span rpcTraceChild3 = spansByParentId.find(rpcTraceChild2.getSpanId())
-            .iterator().next();
-        Assert.assertEquals(0, spansByParentId.find(rpcTraceChild3.getSpanId()).size());
-      }
-    });
-
-    traceCreator.createThreadedTrace();
-    traceCreator.createSimpleTrace();
-    traceCreator.createSampleRpcTrace();
+    Span rpcTraceChild3 = spansByParentId.find(rpcTraceChild2.getSpanId())
+        .iterator().next();
+    Assert.assertEquals(0, spansByParentId.find(rpcTraceChild3.getSpanId()).size());
   }
 
   @Test(timeout=60000)
   public void testRootSpansHaveNonZeroSpanId() throws Exception {
-    TraceScope scope = Trace.startSpan("myRootSpan", new SpanId(100L, 200L));
+    Tracer tracer = new TracerBuilder().
+        name("testRootSpansHaveNonZeroSpanId").
+        tracerPool(new TracerPool("testRootSpansHaveNonZeroSpanId")).
+        conf(HTraceConfiguration.fromKeyValuePairs(
+            "sampler.classes", "AlwaysSampler")).build();
+    TraceScope scope = tracer.
+        newScope("myRootSpan", new SpanId(100L, 200L));
     Assert.assertNotNull(scope);
     Assert.assertEquals("myRootSpan", scope.getSpan().getDescription());
-    Assert.assertEquals(100L, scope.getSpan().getSpanId().getHigh());
     Assert.assertTrue(scope.getSpan().getSpanId().isValid());
+    Assert.assertEquals(100L, scope.getSpan().getSpanId().getHigh());
+    Assert.assertNotEquals(0L, scope.getSpan().getSpanId().getLow());
+    scope.close();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/test/java/org/apache/htrace/core/TestLocalFileSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/test/java/org/apache/htrace/core/TestLocalFileSpanReceiver.java b/htrace-core/src/test/java/org/apache/htrace/core/TestLocalFileSpanReceiver.java
index 90a009a..b4a579b 100644
--- a/htrace-core/src/test/java/org/apache/htrace/core/TestLocalFileSpanReceiver.java
+++ b/htrace-core/src/test/java/org/apache/htrace/core/TestLocalFileSpanReceiver.java
@@ -16,25 +16,21 @@
  */
 package org.apache.htrace.core;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.HashMap;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.junit.Ignore;
 import org.junit.Test;
 
 public class TestLocalFileSpanReceiver {
   @Test
   public void testUniqueLocalTraceFileName() {
     String filename1 = LocalFileSpanReceiver.getUniqueLocalTraceFileName();
-    System.out.println("##### :" + filename1);
     String filename2 = LocalFileSpanReceiver.getUniqueLocalTraceFileName();
-    System.out.println("##### :" + filename2);
     boolean eq = filename1.equals(filename2);
     if (System.getProperty("os.name").startsWith("Linux")) {
       // ${java.io.tmpdir}/[pid]
@@ -48,23 +44,22 @@ public class TestLocalFileSpanReceiver {
   @Test
   public void testWriteToLocalFile() throws IOException {
     String traceFileName = LocalFileSpanReceiver.getUniqueLocalTraceFileName();
-    HashMap<String, String> confMap = new HashMap<String, String>();
-    confMap.put(LocalFileSpanReceiver.PATH_KEY, traceFileName);
-    confMap.put(SpanReceiverBuilder.SPAN_RECEIVER_CONF_KEY,
-                LocalFileSpanReceiver.class.getName());
-    confMap.put(TracerId.TRACER_ID_KEY, "testTrid");
-    SpanReceiver rcvr =
-        new SpanReceiverBuilder(HTraceConfiguration.fromMap(confMap))
-            .logErrors(false).build();
-    Trace.addReceiver(rcvr);
-    TraceScope ts = Trace.startSpan("testWriteToLocalFile", Sampler.ALWAYS);
-    ts.close();
-    Trace.removeReceiver(rcvr);
-    rcvr.close();
+    Tracer tracer = new TracerBuilder().
+        name("testWriteToLocalFileTracer").
+        tracerPool(new TracerPool("testWriteToLocalFile")).
+        conf(HTraceConfiguration.fromKeyValuePairs(
+            "sampler.classes", "AlwaysSampler",
+            "span.receiver.classes", LocalFileSpanReceiver.class.getName(),
+            "local.file.span.receiver.path", traceFileName,
+            "tracer.id", "%{tname}")).
+        build();
+    TraceScope scope = tracer.newScope("testWriteToLocalFile");
+    scope.close();
+    tracer.close();
 
     ObjectMapper mapper = new ObjectMapper();
     MilliSpan span = mapper.readValue(new File(traceFileName), MilliSpan.class);
     assertEquals("testWriteToLocalFile", span.getDescription());
-    assertEquals("testTrid", span.getTracerId());
+    assertEquals("testWriteToLocalFileTracer", span.getTracerId());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/test/java/org/apache/htrace/core/TestNullScope.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/test/java/org/apache/htrace/core/TestNullScope.java b/htrace-core/src/test/java/org/apache/htrace/core/TestNullScope.java
index 3fa9210..592ce79 100644
--- a/htrace-core/src/test/java/org/apache/htrace/core/TestNullScope.java
+++ b/htrace-core/src/test/java/org/apache/htrace/core/TestNullScope.java
@@ -20,15 +20,24 @@ import org.junit.Assert;
 import org.junit.Test;
 
 public class TestNullScope {
+  private void verifyNullScope(TraceScope scope) {
+    Assert.assertTrue(null == scope.getSpan());
+    Assert.assertFalse(scope.detached);
+    scope.detach();
+    Assert.assertTrue(scope.detached);
+    scope.reattach();
+    Assert.assertFalse(scope.detached);
+    scope.close();
+  }
+
   @Test
   public void testNullScope() {
-    Assert.assertTrue(!Trace.isTracing());
-    TraceScope tc = Trace.startSpan("NullScopeSingleton");
-    Assert.assertTrue(tc == NullScope.INSTANCE);
-    tc.detach();
-    tc.detach(); // should not fail even if called multiple times.
-    Assert.assertFalse(tc.isDetached());
-    tc.close();
-    tc.close(); // should not fail even if called multiple times.
+    Tracer tracer = new TracerBuilder().
+        name("testNullScope").
+        tracerPool(new TracerPool("testNullScope")).
+        conf(HTraceConfiguration.EMPTY).
+        build();
+    verifyNullScope(tracer.newScope("testNullScope"));
+    verifyNullScope(tracer.newNullScope());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/test/java/org/apache/htrace/core/TestSampler.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/test/java/org/apache/htrace/core/TestSampler.java b/htrace-core/src/test/java/org/apache/htrace/core/TestSampler.java
index e11799b..aa57d2c 100644
--- a/htrace-core/src/test/java/org/apache/htrace/core/TestSampler.java
+++ b/htrace-core/src/test/java/org/apache/htrace/core/TestSampler.java
@@ -16,40 +16,85 @@
  */
 package org.apache.htrace.core;
 
-import java.util.HashMap;
-import java.util.Map;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
 
 import org.junit.Assert;
 import org.junit.Test;
 
 public class TestSampler {
-  @Test
-  public void testSamplerBuilder() {
-    Sampler alwaysSampler = new SamplerBuilder(
-        HTraceConfiguration.fromKeyValuePairs("sampler", "AlwaysSampler")).
+  private Sampler[] getSamplersFromConf(HTraceConfiguration conf) {
+    Tracer tracer = new TracerBuilder().
+        name("MyTracer").
+        tracerPool(new TracerPool("getSamplersFromConf")).
+        conf(conf).
         build();
-    Assert.assertEquals(AlwaysSampler.class, alwaysSampler.getClass());
+    Sampler[] samplers = tracer.getSamplers();
+    tracer.close();
+    return samplers;
+  }
 
-    Sampler neverSampler = new SamplerBuilder(
-        HTraceConfiguration.fromKeyValuePairs("sampler", "NeverSampler")).
-        build();
-    Assert.assertEquals(NeverSampler.class, neverSampler.getClass());
+  private void checkArrayContains(List<Class<? extends Sampler>> expected,
+                                  Sampler[] samplers) {
+    for (Iterator<Class<? extends Sampler>> iter = expected.iterator();
+         iter.hasNext(); ) {
+      Class<? extends Sampler> samplerClass = iter.next();
+      boolean found = false;
+      for (int i = 0; i < samplers.length; i++) {
+        if (samplers[i] != null) {
+          if (samplers[i].getClass().equals(samplerClass)) {
+            samplers[i] = null;
+            found = true;
+            break;
+          }
+        }
+      }
+      Assert.assertTrue("Failed to find sampler class " +
+          samplerClass.getName(), found);
+    }
+    for (int i = 0; i < samplers.length; i++) {
+      if (samplers[i] != null) {
+        Assert.fail("Got extra sampler of type " +
+            samplers.getClass().getName());
+      }
+    }
+  }
 
-    Sampler neverSampler2 = new SamplerBuilder(HTraceConfiguration.
-        fromKeyValuePairs("sampler", "NonExistentSampler")).
-        build();
-    Assert.assertEquals(NeverSampler.class, neverSampler2.getClass());
+  private void checkArrayContains(Class<? extends Sampler> expected, Sampler[] samplers) {
+    LinkedList<Class<? extends Sampler>> expectedList =
+        new LinkedList<Class<? extends Sampler>>();
+    expectedList.add(expected);
+    checkArrayContains(expectedList, samplers);
+  }
 
-    Sampler neverSampler3 = new SamplerBuilder(HTraceConfiguration.
-        fromKeyValuePairs("sampler.is.not.defined", "NonExistentSampler")).
-        build();
-    Assert.assertEquals(NeverSampler.class, neverSampler3.getClass());
+  @Test
+  public void testTracerBuilderCreatesCorrectSamplers() {
+    Sampler[] samplers = getSamplersFromConf(HTraceConfiguration.
+        fromKeyValuePairs("sampler.classes", "AlwaysSampler"));
+    checkArrayContains(AlwaysSampler.class, samplers);
+
+    samplers = getSamplersFromConf(HTraceConfiguration.
+        fromKeyValuePairs("sampler.classes", "NeverSampler"));
+    checkArrayContains(NeverSampler.class, samplers);
+
+    samplers = getSamplersFromConf(HTraceConfiguration.
+        fromKeyValuePairs("sampler.classes", "NonExistentSampler"));
+    Assert.assertEquals(0, samplers.length);
+
+    samplers = getSamplersFromConf(HTraceConfiguration.EMPTY);
+    Assert.assertEquals(0, samplers.length);
   }
 
   @Test
   public void testAlwaysSampler() {
-    TraceScope cur = Trace.startSpan("test");
-    Assert.assertNotNull(cur);
-    cur.close();
+    AlwaysSampler sampler = new AlwaysSampler(HTraceConfiguration.EMPTY);
+    Assert.assertTrue(sampler.next());
+  }
+
+  @Test
+  public void testNeverSampler() {
+    NeverSampler sampler = new NeverSampler(HTraceConfiguration.EMPTY);
+    Assert.assertTrue(!sampler.next());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/test/java/org/apache/htrace/core/TestSpanReceiverBuilder.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/test/java/org/apache/htrace/core/TestSpanReceiverBuilder.java b/htrace-core/src/test/java/org/apache/htrace/core/TestSpanReceiverBuilder.java
index 79795e4..e0ce2ac 100644
--- a/htrace-core/src/test/java/org/apache/htrace/core/TestSpanReceiverBuilder.java
+++ b/htrace-core/src/test/java/org/apache/htrace/core/TestSpanReceiverBuilder.java
@@ -16,93 +16,87 @@
  */
 package org.apache.htrace.core;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+
 public class TestSpanReceiverBuilder {
   private static final Log LOG =
       LogFactory.getLog(TestSpanReceiverBuilder.class);
 
-  /**
-   * Test that if no span receiver is configured, the builder returns null.
-   */
-  @Test
-  public void testGetNullSpanReceiver() {
-    SpanReceiverBuilder builder =
-        new SpanReceiverBuilder(HTraceConfiguration.EMPTY).logErrors(false);
-    SpanReceiver rcvr = builder.build();
-    Assert.assertEquals(null, rcvr);
-  }
-
-  private static SpanReceiver createSpanReceiver(Map<String, String> m) {
-    HTraceConfiguration hconf = HTraceConfiguration.fromMap(m);
-    SpanReceiverBuilder builder =
-        new SpanReceiverBuilder(hconf).
-            logErrors(false);
-    return builder.build();
+  private List<SpanReceiver> createSpanReceivers(String classes) {
+    Tracer tracer = new TracerBuilder().
+        name("MyTracer").
+        tracerPool(new TracerPool("createSpanReceivers")).
+        conf(HTraceConfiguration.fromKeyValuePairs(
+            "span.receiver.classes", classes)).
+        build();
+    SpanReceiver[] receivers = tracer.getTracerPool().getReceivers();
+    tracer.close();
+    LinkedList<SpanReceiver> receiverList = new LinkedList<SpanReceiver>();
+    for (SpanReceiver item: receivers) {
+      receiverList.add(item);
+    }
+    return receiverList;
   }
 
-  private static final File TMPDIR =
-      new File(System.getProperty("java.io.tmpdir"));
-
-  /**
-   * Test getting various SpanReceiver objects.
-   */
   @Test
-  public void testGetSpanReceivers() throws Exception {
-    HashMap<String, String> confMap = new HashMap<String, String>();
-
-    // Create LocalFileSpanReceiver
-    File testFile = new File(TMPDIR, UUID.randomUUID().toString());
-    try {
-      confMap.put(LocalFileSpanReceiver.PATH_KEY, testFile.getAbsolutePath());
-      confMap.put(SpanReceiverBuilder.SPAN_RECEIVER_CONF_KEY,
-          "org.apache.htrace.core.LocalFileSpanReceiver");
-      SpanReceiver rcvr = createSpanReceiver(confMap);
-      Assert.assertNotNull(rcvr);
-      Assert.assertEquals("org.apache.htrace.core.LocalFileSpanReceiver",
-          rcvr.getClass().getName());
-      rcvr.close();
-    } finally {
-      if (!testFile.delete()) {
-        LOG.debug("failed to delete " + testFile); // keep findbugs happy
+  public void TestCreateStandardSpanReceivers() {
+    List<SpanReceiver> receivers;
+    receivers = createSpanReceivers("");
+    Assert.assertTrue(receivers.isEmpty());
+    receivers = createSpanReceivers("POJOSpanReceiver");
+    Assert.assertTrue(receivers.get(0).getClass().getName().
+        equals("org.apache.htrace.core.POJOSpanReceiver"));
+    receivers = createSpanReceivers(
+               "org.apache.htrace.core.StandardOutSpanReceiver");
+    Assert.assertTrue(receivers.get(0).getClass().getName().
+        equals("org.apache.htrace.core.StandardOutSpanReceiver"));
+    receivers = createSpanReceivers(
+               "POJOSpanReceiver;StandardOutSpanReceiver");
+    Assert.assertEquals(2, receivers.size());
+    for (Iterator<SpanReceiver> iter = receivers.iterator(); iter.hasNext();) {
+      SpanReceiver receiver = iter.next();
+      if (receiver.getClass().getName().equals(
+          "org.apache.htrace.core.POJOSpanReceiver")) {
+        iter.remove();
+        break;
       }
     }
+    for (Iterator<SpanReceiver> iter = receivers.iterator(); iter.hasNext();) {
+      SpanReceiver receiver = iter.next();
+      if (receiver.getClass().getName().equals(
+          "org.apache.htrace.core.StandardOutSpanReceiver")) {
+        iter.remove();
+        break;
+      }
+    }
+    Assert.assertEquals(0, receivers.size());
+  }
 
-    // Create POJOSpanReceiver
-    confMap.remove(LocalFileSpanReceiver.PATH_KEY);
-    confMap.put(SpanReceiverBuilder.SPAN_RECEIVER_CONF_KEY, "POJOSpanReceiver");
-    SpanReceiver rcvr = createSpanReceiver(confMap);
-    Assert.assertEquals("org.apache.htrace.core.POJOSpanReceiver",
-        rcvr.getClass().getName());
-    rcvr.close();
+  public static class GoodSpanReceiver extends SpanReceiver {
+    public GoodSpanReceiver(HTraceConfiguration conf) {
+    }
 
-    // Create StandardOutSpanReceiver
-    confMap.remove(LocalFileSpanReceiver.PATH_KEY);
-    confMap.put(SpanReceiverBuilder.SPAN_RECEIVER_CONF_KEY,
-        "org.apache.htrace.core.StandardOutSpanReceiver");
-    rcvr = createSpanReceiver(confMap);
-    Assert.assertEquals("org.apache.htrace.core.StandardOutSpanReceiver",
-        rcvr.getClass().getName());
-    rcvr.close();
-  }
+    @Override
+    public void receiveSpan(Span span) {
+    }
 
-  public static class TestSpanReceiver implements SpanReceiver {
-    final static String SUCCEEDS = "test.span.receiver.succeeds";
+    @Override
+    public void close() throws IOException {
+    }
+  }
 
-    public TestSpanReceiver(HTraceConfiguration conf) {
-      if (conf.get(SUCCEEDS) == null) {
-        throw new RuntimeException("Can't create TestSpanReceiver: " +
-            "invalid configuration.");
-      }
+  public static class BadSpanReceiver extends SpanReceiver {
+    public BadSpanReceiver(HTraceConfiguration conf) {
+      throw new RuntimeException("Can't create BadSpanReceiver");
     }
 
     @Override
@@ -120,20 +114,14 @@ public class TestSpanReceiverBuilder {
    */
   @Test
   public void testGetSpanReceiverWithConstructorError() throws Exception {
-    HashMap<String, String> confMap = new HashMap<String, String>();
-
-    // Create TestSpanReceiver
-    confMap.put(SpanReceiverBuilder.SPAN_RECEIVER_CONF_KEY,
-        TestSpanReceiver.class.getName());
-    confMap.put(TestSpanReceiver.SUCCEEDS, "true");
-    SpanReceiver rcvr = createSpanReceiver(confMap);
-    Assert.assertEquals(TestSpanReceiver.class.getName(),
-        rcvr.getClass().getName());
-    rcvr.close();
-
-    // Fail to create TestSpanReceiver
-    confMap.remove(TestSpanReceiver.SUCCEEDS);
-    rcvr = createSpanReceiver(confMap);
-    Assert.assertEquals(null, rcvr);
+    List<SpanReceiver> receivers;
+    receivers = createSpanReceivers(
+        GoodSpanReceiver.class.getName());
+    Assert.assertEquals(1, receivers.size());
+    Assert.assertTrue(receivers.get(0).getClass().getName().
+        contains("GoodSpanReceiver"));
+    receivers = createSpanReceivers(
+        BadSpanReceiver.class.getName());
+    Assert.assertEquals(0, receivers.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/test/java/org/apache/htrace/core/TestTracerId.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/test/java/org/apache/htrace/core/TestTracerId.java b/htrace-core/src/test/java/org/apache/htrace/core/TestTracerId.java
index ac43653..1e842c5 100644
--- a/htrace-core/src/test/java/org/apache/htrace/core/TestTracerId.java
+++ b/htrace-core/src/test/java/org/apache/htrace/core/TestTracerId.java
@@ -16,13 +16,17 @@
  */
 package org.apache.htrace.core;
 
+import static org.junit.Assert.assertEquals;
+
 import java.io.IOException;
+
 import org.junit.Test;
-import static org.junit.Assert.assertEquals;
 
 public class TestTracerId {
   private void testTracerIdImpl(String expected, String fmt) {
-    assertEquals(expected, new TracerId(fmt).get());
+    assertEquals(expected, new TracerId(
+        HTraceConfiguration.fromKeyValuePairs(TracerId.TRACER_ID_KEY, fmt),
+        "TracerName").get());
   }
 
   @Test
@@ -36,12 +40,13 @@ public class TestTracerId {
 
   @Test
   public void testSubstitutionVariables() throws IOException {
-    testTracerIdImpl(TracerId.getProcessName(), "${pname}");
-    testTracerIdImpl("my." + TracerId.getProcessName(), "my.${pname}");
-    testTracerIdImpl(TracerId.getBestIpString() + ".str", "${ip}.str");
-    testTracerIdImpl("${pname}", "\\${pname}");
-    testTracerIdImpl("$cash$money{}", "$cash$money{}");
+    testTracerIdImpl("myTracerName", "my%{tname}");
+    testTracerIdImpl(TracerId.getProcessName(), "%{pname}");
+    testTracerIdImpl("my." + TracerId.getProcessName(), "my.%{pname}");
+    testTracerIdImpl(TracerId.getBestIpString() + ".str", "%{ip}.str");
+    testTracerIdImpl("%{pname}", "\\%{pname}");
+    testTracerIdImpl("%cash%money{}", "%cash%money{}");
     testTracerIdImpl("Foo." + Long.valueOf(TracerId.getOsPid()).toString(),
-        "Foo.${pid}");
+        "Foo.%{pid}");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/test/java/org/apache/htrace/core/TraceCreator.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/test/java/org/apache/htrace/core/TraceCreator.java b/htrace-core/src/test/java/org/apache/htrace/core/TraceCreator.java
index f6ae97f..b843999 100644
--- a/htrace-core/src/test/java/org/apache/htrace/core/TraceCreator.java
+++ b/htrace-core/src/test/java/org/apache/htrace/core/TraceCreator.java
@@ -16,54 +16,25 @@
  */
 package org.apache.htrace.core;
 
-import org.junit.rules.TestRule;
-import org.junit.runner.Description;
-import org.junit.runners.model.Statement;
-
-import java.util.Collection;
 import java.util.Random;
 import java.util.concurrent.ThreadLocalRandom;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
 
 /**
  * Does some stuff and traces it.
  */
-public class TraceCreator implements TestRule {
-  private final List<SpanReceiver> receivers = new ArrayList<SpanReceiver>();
-
+public class TraceCreator {
   public static final String RPC_TRACE_ROOT = "createSampleRpcTrace";
   public static final String THREADED_TRACE_ROOT = "createThreadedTrace";
   public static final String SIMPLE_TRACE_ROOT = "createSimpleTrace";
 
-  public TraceCreator addReceiver(SpanReceiver receiver) {
-    Trace.addReceiver(receiver);
-    this.receivers.add(receiver);
-    return this;
-  }
+  private final Tracer tracer;
 
-  @Override
-  public Statement apply(final Statement base, Description description) {
-    return new Statement() {
-      @Override
-      public void evaluate() throws Throwable {
-        try {
-          base.evaluate();
-          for (SpanReceiver receiver : receivers) {
-            receiver.close();
-          }
-        } finally {
-          for (SpanReceiver receiver : receivers) {
-            Trace.removeReceiver(receiver);
-          }
-        }
-      }
-    };
+  public TraceCreator(Tracer tracer) {
+    this.tracer = tracer;
   }
 
   public void createSampleRpcTrace() {
-    TraceScope s = Trace.startSpan(RPC_TRACE_ROOT, Sampler.ALWAYS);
+    TraceScope s = tracer.newScope(RPC_TRACE_ROOT);
     try {
       pretendRpcSend();
     } finally {
@@ -72,7 +43,7 @@ public class TraceCreator implements TestRule {
   }
 
   public void createSimpleTrace() {
-    TraceScope s = Trace.startSpan(SIMPLE_TRACE_ROOT, Sampler.ALWAYS);
+    TraceScope s = tracer.newScope(SIMPLE_TRACE_ROOT);
     try {
       importantWork1();
     } finally {
@@ -84,14 +55,14 @@ public class TraceCreator implements TestRule {
    * Creates the demo trace (will create different traces from call to call).
    */
   public void createThreadedTrace() {
-    TraceScope s = Trace.startSpan(THREADED_TRACE_ROOT, Sampler.ALWAYS);
+    TraceScope s = tracer.newScope(THREADED_TRACE_ROOT);
     try {
       Random r = ThreadLocalRandom.current();
       int numThreads = r.nextInt(4) + 1;
       Thread[] threads = new Thread[numThreads];
 
       for (int i = 0; i < numThreads; i++) {
-        threads[i] = new Thread(Trace.wrap(new MyRunnable()));
+        threads[i] = new Thread(tracer.wrap(new MyRunnable(), null));
       }
       for (int i = 0; i < numThreads; i++) {
         threads[i].start();
@@ -109,7 +80,7 @@ public class TraceCreator implements TestRule {
   }
 
   private void importantWork1() {
-    TraceScope cur = Trace.startSpan("important work 1");
+    TraceScope cur = tracer.newScope("important work 1");
     try {
       Thread.sleep((long) (2000 * Math.random()));
       importantWork2();
@@ -121,7 +92,7 @@ public class TraceCreator implements TestRule {
   }
 
   private void importantWork2() {
-    TraceScope cur = Trace.startSpan("important work 2");
+    TraceScope cur = tracer.newScope("important work 2");
     try {
       Thread.sleep((long) (2000 * Math.random()));
     } catch (InterruptedException e) {
@@ -142,7 +113,7 @@ public class TraceCreator implements TestRule {
       } catch (InterruptedException ie) {
         Thread.currentThread().interrupt();
       } catch (ArithmeticException ae) {
-        TraceScope c = Trace.startSpan("dealing with arithmetic exception.");
+        TraceScope c = tracer.newScope("dealing with arithmetic exception.");
         try {
           Thread.sleep((long) (3000 * Math.random()));
         } catch (InterruptedException ie1) {
@@ -155,11 +126,12 @@ public class TraceCreator implements TestRule {
   }
 
   public void pretendRpcSend() {
-    pretendRpcReceiveWithTraceInfo(Trace.currentSpan());
+    Span span = tracer.getCurrentSpan();
+    pretendRpcReceiveWithTraceInfo(span.getSpanId());
   }
 
-  public void pretendRpcReceiveWithTraceInfo(Span parent) {
-    TraceScope s = Trace.startSpan("received RPC", parent);
+  public void pretendRpcReceiveWithTraceInfo(SpanId parentId) {
+    TraceScope s = tracer.newScope("received RPC", parentId);
     try {
       importantWork1();
     } finally {

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-flume/src/main/java/org/apache/htrace/impl/FlumeSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-flume/src/main/java/org/apache/htrace/impl/FlumeSpanReceiver.java b/htrace-flume/src/main/java/org/apache/htrace/impl/FlumeSpanReceiver.java
index 4820bd0..199f93d 100644
--- a/htrace-flume/src/main/java/org/apache/htrace/impl/FlumeSpanReceiver.java
+++ b/htrace-flume/src/main/java/org/apache/htrace/impl/FlumeSpanReceiver.java
@@ -44,7 +44,7 @@ import org.apache.htrace.core.Span;
 import org.apache.htrace.core.SpanReceiver;
 import org.apache.htrace.core.TracerId;
 
-public class FlumeSpanReceiver implements SpanReceiver {
+public class FlumeSpanReceiver extends SpanReceiver {
   private static final Log LOG = LogFactory.getLog(FlumeSpanReceiver.class);
 
   public static final String NUM_THREADS_KEY = "htrace.flume.num-threads";
@@ -90,12 +90,10 @@ public class FlumeSpanReceiver implements SpanReceiver {
   private int maxSpanBatchSize;
   private String flumeHostName;
   private int flumePort;
-  private final TracerId tracerId;
 
   public FlumeSpanReceiver(HTraceConfiguration conf) {
     this.queue = new ArrayBlockingQueue<Span>(1000);
     this.tf = new SimpleThreadFactory();
-    this.tracerId = new TracerId(conf);
     configure(conf);
   }
 
@@ -274,9 +272,6 @@ public class FlumeSpanReceiver implements SpanReceiver {
   public void receiveSpan(Span span) {
     if (running.get()) {
       try {
-        if (span.getTracerId().isEmpty()) {
-          span.setTracerId(tracerId.get());
-        }
         this.queue.add(span);
       } catch (IllegalStateException e) {
         LOG.error("Error trying to append span (" +

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-flume/src/test/java/org/apache/htrace/impl/TestFlumeSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-flume/src/test/java/org/apache/htrace/impl/TestFlumeSpanReceiver.java b/htrace-flume/src/test/java/org/apache/htrace/impl/TestFlumeSpanReceiver.java
index d5037c8..6cdab2e 100644
--- a/htrace-flume/src/test/java/org/apache/htrace/impl/TestFlumeSpanReceiver.java
+++ b/htrace-flume/src/test/java/org/apache/htrace/impl/TestFlumeSpanReceiver.java
@@ -17,12 +17,17 @@
 
 package org.apache.htrace.impl;
 
-import org.apache.htrace.core.HTraceConfiguration;
+import org.apache.htrace.core.AlwaysSampler;
 import org.apache.htrace.core.HTraceConfiguration;
 import org.apache.htrace.core.MilliSpan;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.SpanId;
 import org.apache.htrace.core.TraceCreator;
+import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
+import org.apache.htrace.core.TracerBuilder;
+import org.apache.htrace.core.TracerPool;
+
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -31,56 +36,55 @@ import java.io.IOException;
 
 public class TestFlumeSpanReceiver {
   @Rule
-  public TraceCreator traceCreator = new TraceCreator();
-  @Rule
   public FakeFlume flumeServer = new FakeFlume();
 
-  @Test
-  public void testSimpleTraces() throws IOException, InterruptedException {
-    traceCreator.addReceiver(new FlumeSpanReceiver(
-        HTraceConfiguration.fromKeyValuePairs(
-            FlumeSpanReceiver.FLUME_PORT_KEY, Integer.toString(flumeServer.getPort())
-        )
-    ));
+  private Tracer newTracer() {
+    return new TracerBuilder().
+        name("FlumeTracer").
+        tracerPool(new TracerPool("newTracer")).
+        conf(HTraceConfiguration.fromKeyValuePairs(
+            FlumeSpanReceiver.FLUME_PORT_KEY,
+            Integer.toString(flumeServer.getPort()),
+            "span.receiver.classes", FlumeSpanReceiver.class.getName(),
+            "sampler.classes", AlwaysSampler.class.getName()
+        )).build();
+  }
 
+  @Test(timeout=120000)
+  public void testSimpleTraces() throws IOException, InterruptedException {
+    Tracer tracer = newTracer();
     Span rootSpan = new MilliSpan.Builder().
         description("root").
         spanId(new SpanId(100, 100)).
         tracerId("test").
         begin(System.currentTimeMillis()).
         build();
-    Span innerOne = rootSpan.child("Some good work");
-    Span innerTwo = innerOne.child("Some more good work");
-    innerTwo.stop();
-    Assert.assertTrue(flumeServer.nextEventBodyAsString().contains(innerTwo.getDescription()));
-    innerOne.stop();
-    Assert.assertTrue(flumeServer.nextEventBodyAsString().contains(innerOne.getDescription()));
+    TraceScope rootScope = tracer.newScope("root");
+    TraceScope innerOne = tracer.newScope("innerOne");
+    TraceScope innerTwo = tracer.newScope("innerTwo");
+    innerTwo.close();
+    Assert.assertTrue(flumeServer.nextEventBodyAsString().contains("innerTwo"));
+    innerOne.close();
+    Assert.assertTrue(flumeServer.nextEventBodyAsString().contains("innerOne"));
     rootSpan.addKVAnnotation("foo", "bar");
     rootSpan.addTimelineAnnotation("timeline");
-    rootSpan.stop();
-    Assert.assertTrue(flumeServer.nextEventBodyAsString().contains(rootSpan.getDescription()));
+    rootScope.close();
+    Assert.assertTrue(flumeServer.nextEventBodyAsString().contains("root"));
+    tracer.close();
   }
 
-  @Test
+  @Test(timeout=120000)
   public void testConcurrency() throws IOException {
-    traceCreator.addReceiver(new FlumeSpanReceiver(
-        HTraceConfiguration.fromKeyValuePairs(
-            FlumeSpanReceiver.FLUME_PORT_KEY, Integer.toString(flumeServer.getPort())
-        )
-    ));
-
+    Tracer tracer = newTracer();
+    TraceCreator traceCreator = new TraceCreator(tracer);
     flumeServer.alwaysOk();
     traceCreator.createThreadedTrace();
   }
 
-  @Test
+  @Test(timeout=120000)
   public void testResilience() throws IOException {
-    traceCreator.addReceiver(new FlumeSpanReceiver(
-        HTraceConfiguration.fromKeyValuePairs(
-            FlumeSpanReceiver.FLUME_PORT_KEY, Integer.toString(flumeServer.getPort())
-        )
-    ));
-
+    Tracer tracer = newTracer();
+    TraceCreator traceCreator = new TraceCreator(tracer);
     flumeServer.alwaysFail();
     traceCreator.createThreadedTrace();
   }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-hbase/src/main/java/org/apache/htrace/HBaseSpanReceiverHost.java
----------------------------------------------------------------------
diff --git a/htrace-hbase/src/main/java/org/apache/htrace/HBaseSpanReceiverHost.java b/htrace-hbase/src/main/java/org/apache/htrace/HBaseSpanReceiverHost.java
deleted file mode 100644
index 12f36cb..0000000
--- a/htrace-hbase/src/main/java/org/apache/htrace/HBaseSpanReceiverHost.java
+++ /dev/null
@@ -1,107 +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.htrace;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashSet;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.htrace.core.HTraceConfiguration;
-import org.apache.htrace.core.SpanReceiver;
-import org.apache.htrace.core.SpanReceiverBuilder;
-import org.apache.htrace.core.Trace;
-
-/**
- * This class provides functions for reading the names of SpanReceivers from
- * hbase-site.xml, adding those SpanReceivers to the Tracer, and closing those
- * SpanReceivers when appropriate.
- */
-public class HBaseSpanReceiverHost {
-  public static final String SPAN_RECEIVERS_CONF_KEY = "hbase.trace.spanreceiver.classes";
-  private static final Log LOG = LogFactory.getLog(HBaseSpanReceiverHost.class);
-  private Collection<SpanReceiver> receivers;
-  private Configuration conf;
-  private boolean closed = false;
-
-  private static enum SingletonHolder {
-    INSTANCE;
-    Object lock = new Object();
-    HBaseSpanReceiverHost host = null;
-  }
-
-  public static HBaseSpanReceiverHost getInstance(Configuration conf) {
-    synchronized (SingletonHolder.INSTANCE.lock) {
-      if (SingletonHolder.INSTANCE.host != null) {
-        return SingletonHolder.INSTANCE.host;
-      }
-
-      HBaseSpanReceiverHost host = new HBaseSpanReceiverHost(conf);
-      host.loadSpanReceivers();
-      SingletonHolder.INSTANCE.host = host;
-      return SingletonHolder.INSTANCE.host;
-    }
-
-  }
-
-  HBaseSpanReceiverHost(Configuration conf) {
-    receivers = new HashSet<SpanReceiver>();
-    this.conf = conf;
-  }
-
-  /**
-   * Reads the names of classes specified in the
-   * "hbase.trace.spanreceiver.classes" property and instantiates and registers
-   * them with the Tracer as SpanReceiver's.
-   *
-   */
-  public void loadSpanReceivers() {
-    String[] receiverNames = conf.getStrings(SPAN_RECEIVERS_CONF_KEY);
-    if (receiverNames == null || receiverNames.length == 0) {
-      return;
-    }
-    SpanReceiverBuilder builder = new SpanReceiverBuilder(new HBaseHTraceConfiguration(this.conf));
-    for (String className : receiverNames) {
-      SpanReceiver receiver = builder.spanReceiverClass(className.trim()).build();
-      if (receiver != null) {
-        receivers.add(receiver);
-        LOG.info("SpanReceiver " + className + " was loaded successfully.");
-      }
-    }
-    for (SpanReceiver rcvr : receivers) {
-      Trace.addReceiver(rcvr);
-    }
-  }
-
-  /**
-   * Calls close() on all SpanReceivers created by this HBaseSpanReceiverHost.
-   */
-  public synchronized void closeReceivers() {
-    if (closed) return;
-    closed = true;
-    for (SpanReceiver rcvr : receivers) {
-      try {
-        rcvr.close();
-      } catch (IOException e) {
-        LOG.warn("Unable to close SpanReceiver correctly: " + e.getMessage(), e);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-hbase/src/main/java/org/apache/htrace/impl/HBaseSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-hbase/src/main/java/org/apache/htrace/impl/HBaseSpanReceiver.java b/htrace-hbase/src/main/java/org/apache/htrace/impl/HBaseSpanReceiver.java
index 1f54f18..85de849 100644
--- a/htrace-hbase/src/main/java/org/apache/htrace/impl/HBaseSpanReceiver.java
+++ b/htrace-hbase/src/main/java/org/apache/htrace/impl/HBaseSpanReceiver.java
@@ -45,11 +45,10 @@ import org.apache.htrace.core.HTraceConfiguration;
 import org.apache.htrace.core.Sampler;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.SpanReceiver;
-import org.apache.htrace.core.SpanReceiverBuilder;
 import org.apache.htrace.core.TimelineAnnotation;
-import org.apache.htrace.core.Trace;
-import org.apache.htrace.core.TracerId;
 import org.apache.htrace.core.TraceScope;
+import org.apache.htrace.core.Tracer;
+import org.apache.htrace.core.TracerBuilder;
 import org.apache.htrace.protobuf.generated.SpanProtos;
 
 /**
@@ -59,7 +58,7 @@ import org.apache.htrace.protobuf.generated.SpanProtos;
  * From there background worker threads will send them
  * to a HBase database.
  */
-public class HBaseSpanReceiver implements SpanReceiver {
+public class HBaseSpanReceiver extends SpanReceiver {
   private static final Log LOG = LogFactory.getLog(HBaseSpanReceiver.class);
 
   public static final String COLLECTOR_QUORUM_KEY = "htrace.hbase.collector-quorum";
@@ -127,7 +126,6 @@ public class HBaseSpanReceiver implements SpanReceiver {
   private final byte[] cf;
   private final byte[] icf;
   private final int maxSpanBatchSize;
-  private final TracerId tracerId;
 
   public HBaseSpanReceiver(HTraceConfiguration conf) {
     this.queue = new ArrayBlockingQueue<Span>(1000);
@@ -155,7 +153,6 @@ public class HBaseSpanReceiver implements SpanReceiver {
     for (int i = 0; i < numThreads; i++) {
       this.service.submit(new WriteSpanRunnable());
     }
-    this.tracerId = new TracerId(conf);
   }
 
   private class WriteSpanRunnable implements Runnable {
@@ -334,9 +331,6 @@ public class HBaseSpanReceiver implements SpanReceiver {
   public void receiveSpan(Span span) {
     if (running.get()) {
       try {
-        if (span.getTracerId().isEmpty()) {
-          span.setTracerId(tracerId.get());
-        }
         this.queue.add(span);
       } catch (IllegalStateException e) {
         // todo: supress repeating error logs.
@@ -354,29 +348,28 @@ public class HBaseSpanReceiver implements SpanReceiver {
    * @throws IOException
    */
   public static void main(String[] args) throws Exception {
-    SpanReceiverBuilder builder =
-      new SpanReceiverBuilder(new HBaseHTraceConfiguration(HBaseConfiguration.create()));
-    SpanReceiver receiver =
-      builder.spanReceiverClass(HBaseSpanReceiver.class.getName()).build();
-    Trace.addReceiver(receiver);
-    TraceScope parent = Trace.startSpan("HBaseSpanReceiver.main.parent", Sampler.ALWAYS);
+    Tracer tracer = new TracerBuilder().
+        conf(new HBaseHTraceConfiguration(HBaseConfiguration.create())).
+        build();
+    tracer.addSampler(Sampler.ALWAYS);
+    TraceScope parent = tracer.newScope("HBaseSpanReceiver.main.parent");
     Thread.sleep(10);
     long traceid = parent.getSpan().getSpanId().getHigh();
-    TraceScope child1 = Trace.startSpan("HBaseSpanReceiver.main.child.1");
+    TraceScope child1 = tracer.newScope("HBaseSpanReceiver.main.child.1");
     Thread.sleep(10);
     child1.close();
-    TraceScope child2 = Trace.startSpan("HBaseSpanReceiver.main.child.2", parent.getSpan());
+    TraceScope child2 = tracer.newScope("HBaseSpanReceiver.main.child.2");
     Thread.sleep(10);
-    TraceScope gchild = Trace.startSpan("HBaseSpanReceiver.main.grandchild");
-    Trace.addTimelineAnnotation("annotation 1.");
+    TraceScope gchild = tracer.newScope("HBaseSpanReceiver.main.grandchild");
+    gchild.addTimelineAnnotation("annotation 1.");
     Thread.sleep(10);
-    Trace.addTimelineAnnotation("annotation 2.");
+    gchild.addTimelineAnnotation("annotation 2.");
     gchild.close();
     Thread.sleep(10);
     child2.close();
     Thread.sleep(10);
     parent.close();
-    receiver.close();
+    tracer.close();
     System.out.println("trace id: " + traceid);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-hbase/src/test/java/org/apache/htrace/impl/HBaseTestUtil.java
----------------------------------------------------------------------
diff --git a/htrace-hbase/src/test/java/org/apache/htrace/impl/HBaseTestUtil.java b/htrace-hbase/src/test/java/org/apache/htrace/impl/HBaseTestUtil.java
index db7f6c6..d278c5e 100644
--- a/htrace-hbase/src/test/java/org/apache/htrace/impl/HBaseTestUtil.java
+++ b/htrace-hbase/src/test/java/org/apache/htrace/impl/HBaseTestUtil.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.htrace.HBaseHTraceConfiguration;
 import org.apache.htrace.core.SpanReceiver;
-import org.apache.htrace.core.SpanReceiverBuilder;
 import org.junit.Assert;
 
 
@@ -59,7 +58,8 @@ public class HBaseTestUtil {
   }
 
   public static SpanReceiver startReceiver(Configuration conf) {
-    return new SpanReceiverBuilder(new HBaseHTraceConfiguration(conf)).build();
+    return new SpanReceiver.Builder(
+        new HBaseHTraceConfiguration(conf)).build();
   }
 
   public static SpanReceiver startReceiver(HBaseTestingUtility util) {

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-hbase/src/test/java/org/apache/htrace/impl/TestHBaseSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-hbase/src/test/java/org/apache/htrace/impl/TestHBaseSpanReceiver.java b/htrace-hbase/src/test/java/org/apache/htrace/impl/TestHBaseSpanReceiver.java
index f1a3d19..253a873 100644
--- a/htrace-hbase/src/test/java/org/apache/htrace/impl/TestHBaseSpanReceiver.java
+++ b/htrace-hbase/src/test/java/org/apache/htrace/impl/TestHBaseSpanReceiver.java
@@ -30,20 +30,25 @@ import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.htrace.core.HTraceConfiguration;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.SpanReceiver;
 import org.apache.htrace.core.TimelineAnnotation;
 import org.apache.htrace.core.TraceCreator;
-import org.apache.htrace.core.TraceGraph.SpansByParent;
 import org.apache.htrace.core.TraceGraph;
+import org.apache.htrace.core.TraceGraph.SpansByParent;
+import org.apache.htrace.core.Tracer;
+import org.apache.htrace.core.TracerBuilder;
+import org.apache.htrace.core.TracerPool;
 import org.apache.htrace.protobuf.generated.SpanProtos;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -65,16 +70,39 @@ public class TestHBaseSpanReceiver {
     UTIL.shutdownMiniCluster();
   }
 
+  private Table createTable(HBaseTestingUtility util) {
+    Table htable = null;
+    try {
+      htable = util.createTable(Bytes.toBytes(HBaseSpanReceiver.DEFAULT_TABLE),
+          new byte[][]{Bytes.toBytes(HBaseSpanReceiver.DEFAULT_COLUMNFAMILY),
+              Bytes.toBytes(HBaseSpanReceiver.DEFAULT_INDEXFAMILY)});
+    } catch (IOException e) {
+      Assert.fail("failed to create htrace table. " + e.getMessage());
+    }
+    return htable;
+  }
+
   // Reenable after fix circular dependency
   @Ignore @Test
   public void testHBaseSpanReceiver() {
-    Table htable = HBaseTestUtil.createTable(UTIL);
-    SpanReceiver receiver = HBaseTestUtil.startReceiver(UTIL);
-    TraceCreator tc = new TraceCreator().addReceiver(receiver);
+    Table htable = createTable(UTIL);
+    Configuration conf = UTIL.getConfiguration();
+    Tracer tracer = new TracerBuilder().
+        name("testHBaseSpanReceiver").
+        tracerPool(new TracerPool("testHBaseSpanReceiver")).
+        conf(HTraceConfiguration.fromKeyValuePairs(
+          HBaseSpanReceiver.COLLECTOR_QUORUM_KEY,
+              conf.get(HConstants.ZOOKEEPER_QUORUM),
+          HBaseSpanReceiver.ZOOKEEPER_CLIENT_PORT_KEY,
+              "" + conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, 2181),
+          HBaseSpanReceiver.ZOOKEEPER_ZNODE_PARENT_KEY,
+              conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)
+        )).build();
+    TraceCreator tc = new TraceCreator(tracer);
     tc.createThreadedTrace();
     tc.createSimpleTrace();
     tc.createSampleRpcTrace();
-    HBaseTestUtil.stopReceiver(receiver);
+    tracer.close();
     Scan scan = new Scan();
     scan.addFamily(Bytes.toBytes(HBaseSpanReceiver.DEFAULT_COLUMNFAMILY));
     scan.setMaxVersions(1);

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-htraced/src/main/java/org/apache/htrace/impl/HTracedRESTReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-htraced/src/main/java/org/apache/htrace/impl/HTracedRESTReceiver.java b/htrace-htraced/src/main/java/org/apache/htrace/impl/HTracedRESTReceiver.java
index 77d0f49..1ac5126 100644
--- a/htrace-htraced/src/main/java/org/apache/htrace/impl/HTracedRESTReceiver.java
+++ b/htrace-htraced/src/main/java/org/apache/htrace/impl/HTracedRESTReceiver.java
@@ -32,7 +32,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.htrace.core.HTraceConfiguration;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.SpanReceiver;
-import org.apache.htrace.core.TracerId;
 import org.eclipse.jetty.client.HttpClient;
 import org.eclipse.jetty.client.api.ContentResponse;
 import org.eclipse.jetty.client.api.Request;
@@ -65,7 +64,7 @@ import org.eclipse.jetty.http.HttpStatus;
  * TODO: Add lazy start; don't start background thread till a span gets queued.
  * TODO: Add some metrics; how many times we've run, how many spans and what size we've sent.
  */
-public class HTracedRESTReceiver implements SpanReceiver {
+public class HTracedRESTReceiver extends SpanReceiver {
   private static final Log LOG = LogFactory.getLog(HTracedRESTReceiver.class);
 
   /**
@@ -170,11 +169,6 @@ public class HTracedRESTReceiver implements SpanReceiver {
   private boolean mustStartFlush;
 
   /**
-   * The process ID to use for all spans.
-   */
-  private final TracerId tracerId;
-
-  /**
    * Create an HttpClient instance.
    *
    * @param connTimeout         The timeout to use for connecting.
@@ -226,7 +220,6 @@ public class HTracedRESTReceiver implements SpanReceiver {
             capacity + ", url=" + url +  ", periodInMs=" + periodInMs +
             ", maxToSendAtATime=" + maxToSendAtATime);
     }
-    tracerId = new TracerId(conf);
   }
 
   /**
@@ -322,7 +315,6 @@ public class HTracedRESTReceiver implements SpanReceiver {
       try {
         Request request = httpClient.newRequest(url).method(HttpMethod.POST);
         request.header(HttpHeader.CONTENT_TYPE, "application/json");
-        request.header("htrace-trid", tracerId.get());
         StringBuilder bld = new StringBuilder();
         for (Span span : spanBuf) {
           bld.append(span.toJson());

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-htraced/src/test/java/org/apache/htrace/impl/TestHTracedRESTReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-htraced/src/test/java/org/apache/htrace/impl/TestHTracedRESTReceiver.java b/htrace-htraced/src/test/java/org/apache/htrace/impl/TestHTracedRESTReceiver.java
index 5e4bfc6..d52f071 100644
--- a/htrace-htraced/src/test/java/org/apache/htrace/impl/TestHTracedRESTReceiver.java
+++ b/htrace-htraced/src/test/java/org/apache/htrace/impl/TestHTracedRESTReceiver.java
@@ -133,6 +133,8 @@ public class TestHTracedRESTReceiver {
           spanId(new SpanId(1L, i));
       if (i == NUM_SPANS - 1) {
         builder.tracerId("specialTrid");
+      } else {
+        builder.tracerId(TestHTraceConfiguration.TRACER_ID);
       }
       spans[i] = builder.build();
     }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-zipkin/src/main/java/org/apache/htrace/impl/ZipkinSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-zipkin/src/main/java/org/apache/htrace/impl/ZipkinSpanReceiver.java b/htrace-zipkin/src/main/java/org/apache/htrace/impl/ZipkinSpanReceiver.java
index 6ba1afe..c106fa8 100644
--- a/htrace-zipkin/src/main/java/org/apache/htrace/impl/ZipkinSpanReceiver.java
+++ b/htrace-zipkin/src/main/java/org/apache/htrace/impl/ZipkinSpanReceiver.java
@@ -26,7 +26,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.htrace.core.HTraceConfiguration;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.SpanReceiver;
-import org.apache.htrace.core.TracerId;
 import org.apache.htrace.zipkin.HTraceToZipkinConverter;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TProtocol;
@@ -60,7 +59,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
  * HTrace spans are queued into a blocking queue.  From there background worker threads will
  * batch the spans together and then send them through to a Zipkin collector.
  */
-public class ZipkinSpanReceiver implements SpanReceiver {
+public class ZipkinSpanReceiver extends SpanReceiver {
   private static final Log LOG = LogFactory.getLog(ZipkinSpanReceiver.class);
 
   /**
@@ -141,8 +140,6 @@ public class ZipkinSpanReceiver implements SpanReceiver {
     }
   };
 
-  private final TracerId tracerId;
-
   ////////////////////
   /// Variables that will change on each call to configure()
   ///////////////////
@@ -155,7 +152,6 @@ public class ZipkinSpanReceiver implements SpanReceiver {
   public ZipkinSpanReceiver(HTraceConfiguration conf) {
     this.queue = new ArrayBlockingQueue<Span>(1000);
     this.protocolFactory = new TBinaryProtocol.Factory();
-    this.tracerId = new TracerId(conf);
     configure(conf);
   }
 
@@ -375,9 +371,6 @@ public class ZipkinSpanReceiver implements SpanReceiver {
   public void receiveSpan(Span span) {
     if (running.get()) {
       try {
-        if (span.getTracerId().isEmpty()) {
-          span.setTracerId(tracerId.get());
-        }
         this.queue.add(span);
       } catch (IllegalStateException e) {
         LOG.error("Error trying to append span (" + span.getDescription() + ") to the queue."

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-zipkin/src/test/java/org/apache/htrace/TestHTraceSpanToZipkinSpan.java
----------------------------------------------------------------------
diff --git a/htrace-zipkin/src/test/java/org/apache/htrace/TestHTraceSpanToZipkinSpan.java b/htrace-zipkin/src/test/java/org/apache/htrace/TestHTraceSpanToZipkinSpan.java
index 5b1ac3d..f166d35 100644
--- a/htrace-zipkin/src/test/java/org/apache/htrace/TestHTraceSpanToZipkinSpan.java
+++ b/htrace-zipkin/src/test/java/org/apache/htrace/TestHTraceSpanToZipkinSpan.java
@@ -20,12 +20,10 @@ package org.apache.htrace.zipkin;
 import com.twitter.zipkin.gen.zipkinCoreConstants;
 
 import org.apache.htrace.core.HTraceConfiguration;
-import org.apache.htrace.core.Span;
-import org.apache.htrace.core.SpanId;
-import org.apache.htrace.core.Trace;
 import org.apache.htrace.core.MilliSpan;
 import org.apache.htrace.core.POJOSpanReceiver;
-import org.apache.htrace.zipkin.HTraceToZipkinConverter;
+import org.apache.htrace.core.Span;
+import org.apache.htrace.core.SpanId;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -39,14 +37,10 @@ import static org.junit.Assert.assertTrue;
  * Creates HTrace and then convert it to Zipkin trace and checks whether it is a valid span or not.
  */
 public class TestHTraceSpanToZipkinSpan {
-
   private static final String ROOT_SPAN_DESC = "ROOT";
 
   @Test
   public void testHTraceToZipkin() throws IOException {
-    POJOSpanReceiver psr = new POJOSpanReceiver(HTraceConfiguration.EMPTY);
-    Trace.addReceiver(psr);
-
     Span rootSpan = new MilliSpan.Builder().
             description(ROOT_SPAN_DESC).
             parents(new SpanId[] { } ).

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/src/main/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/src/main/site/markdown/index.md b/src/main/site/markdown/index.md
index 6198472..e8e88fc 100644
--- a/src/main/site/markdown/index.md
+++ b/src/main/site/markdown/index.md
@@ -28,36 +28,50 @@ Before we get into the details, lets review our terminology.  HTrace
 borrows [Dapper's](http://research.google.com/pubs/pub36356.html)
 terminology.
 
-<b>Span:</b> The basic unit of work. For example, sending an RPC is a
-new span, as is sending a response to an RPC.
-Span's are identified by a unique 64-bit ID for the span and another
-64-bit ID for the trace the span is a part of.  Spans also have other
-data, such as descriptions, key-value annotations, the ID of the span
-that caused them, and process ID's (normally IP address).
+<b>Span:</b> The basic unit of work. For example, sending an RPC is a new span,
+as is sending a response to an RPC.  Spans are identified by a unique 128-bit
+ID.  Spans also have other data, such as descriptions, key-value annotations,
+the ID of the span that caused them, and tracer IDs.
 
 Spans are started and stopped, and they keep track of their timing
 information.  Once you create a span, you must stop it at some point
 in the future.
 
-<b>Trace:</b> A set of spans forming a tree-like structure.  For
-example, if you are running a distributed big-data store, a trace
-might be formed by a put request.
+<b>TracerId:</b> Identifies the Tracer object which created a specific Span.
+The TracerId should identify the source of the Span.  "10.0.0.2/DataNode" is a
+typical TracerID.
+
+<b>SpanReceiver:</b> SpanReceivers handle spans once they have been created.
+Typically, Span Receivers send spans to an external data store to be
+analyzed.  HTrace comes with several standard span receivers, such as
+`LocalFileSpanReceiver`.
+
+<b>Sampler:</b> Samplers determine when tracing should be enabled, and when it
+should be disabled.   The goal of HTrace is to trace an entire request.
 
 ### How to add tracing to your application
 To instrument your system you must:
 
-<b>1. Attach additional information to your RPC's.</b>
-In order to create the causal links necessary for a trace, HTrace
-needs to know about the causal
-relationships between spans.  The only information you need to add to
-your RPC's is two 64-bit longs.  If tracing is enabled (Trace.isTracing()
-returns true) when you send an RPC, attach the ID of the current span
-and the ID of the current trace to the message.
-On the receiving end of the RPC, check to see if the message has the
-additional tracing information above.  If it does, start a new span
-with the information given (more on that in a bit).
-
-<b>2. Wrap your thread changes.</b>
+<b>1. Create a Tracer object.</b>
+You can create a Tracer object via the TracerBuilder.
+
+````java
+    Tracer tracer = new TracerBuilder(conf).setName("MyApp").build();
+...
+
+The TracerBuilder will take care of creating the appropriate Sampler and
+SpanReceiver objects, as well as the Tracer itself.   If a SpanReceiver was
+created, we will install a shutdown hook to close it when the JVM shuts down.
+
+<b>2. Attach additional information to your RPCs.</b>
+In order to create the causal links necessary for a trace, HTrace needs to know
+about the causal relationships between spans.  The only information you need to
+add to your RPCs is the 128-bit span ID.  If tracing is enabled when you send an
+RPC, attach the ID of the current span to the message.  On the receiving end of
+the RPC, check to see if the message has a span ID attached.  If it does, start
+a new trace scope with that span as a parent.
+
+<b>3. Wrap your thread changes.</b>
 HTrace stores span information in java's ThreadLocals, which causes
 the trace to be "lost" on thread changes. The only way to prevent
 this is to "wrap" your thread changes. For example, if your code looks
@@ -104,7 +118,7 @@ the computation that you then stop after the computation has
 finished. It might look like this:
 
 ````java
-    Span computationSpan = Trace.startSpan("Expensive computation.");
+    Span computationSpan = tracer.newScope("Expensive computation.");
     try {
         //expensive computation here
     } finally {
@@ -117,110 +131,10 @@ HTrace also supports key-value annotations on a per-trace basis.
 Example:
 
 ````java
-    Trace.currentTrace().addAnnotation("faultyRecordCounter".getBytes(), "1".getBytes());
+    scope.addAnnotation("faultyRecordCounter".getBytes(), "1".getBytes());
 ````
 
-`Trace.currentTrace()` will not return `null` if the current thread is
-not tracing, but instead it will return a `NullSpan`, which does
-nothing on any of its method calls. The takeaway here is you can call
-methods on the `currentTrace()` without fear of NullPointerExceptions.
-
-###Samplers
-`Sampler` is an interface that defines one function:
-
-````java
-    boolean next(T info);
-````
-
-All of the `Trace.startSpan()` methods can take an optional sampler.
-A new span is only created if the sampler's next function returns
-true.  If the Sampler returns false, the `NullSpan` is returned from
-`startSpan()`, so it's safe to call `stop()` or `addAnnotation()` on it.
-As you may have noticed from the `next()` method signature, Sampler is
-parameterized.  The argument to `next()` is whatever piece of
-information you might need for sampling.  See `Sampler.java` for an
-example of this.  If you do not require any additional information,
-then just ignore the parameter.
-HTrace includes  a sampler that always returns true, a
-sampler that always returns false and a sampler returns true some
-percentage of the time (you pass in the percentage as a decimal at construction).
-
-HTrace comes with several standard samplers, including `AlwaysSampler`,
-`NeverSampler`, `ProbabilitySampler`, and `CountSampler`.  An application can
-use the `SamplerBuilder` to create one of these standard samplers based on the
-current configuration.
-
-````java
-  HTraceConfiguration hconf = createMyHTraceConfiguration();
-  Sampler sampler = new SamplerBuilder(hconf).build();
-````
-
-####Trace.startSpan()
-There is a single method to create and start spans: `startSpan()`.
-For the `startSpan()` methods that do not take an explicit Sampler, the
-default Sampler is used.  The default sampler returns true if and only
-if tracing is already on in the current thread.  That means that
-calling `startSpan()` with no explicit Sampler is a good idea when you
-have information that you would like to add to a trace if it's already
-occurring, but is not something you would want to start a whole new
-trace for.
-
-If you are using a sampler that makes use of the `T info` parameter to
-`next()`, just pass in the object as the last argument.  If you leave it
-out, HTrace will pass `null` for you (so make sure your Samplers can
-handle `null`).
-
-Aside from whether or not you pass in an explicit `Sampler`, there are
-other options you have when calling `startSpan()`.
-For the next section I am assuming you are familiar with the options
-for passing in `Samplers` and `info` parameters, so when I say "no
-arguments," I mean no additional arguments other than whatever
-`Sampler`/`info` parameters you deem necessary.
-
-You can call `startSpan()` with no additional arguments.
-In this case, `Trace.java` will start a span if the sampler (explicit
-or default) returns true. If the current span is not the `NullSpan`, the span
-returned will be a child of the current span, otherwise it will start
-a new trace in the current thread (it will be a
-`ProcessRootMilliSpan`). All of the other `startSpan()` methods take some
-parameter describing the parent span of the span to be created. The
-version that takes a parent id will mostly be used when continuing a trace over
-RPC. The receiver of the RPC will check the message for the 128-bit parent trace
-ID and will call `startSpan()` if it is attached.  The last `startSpan()` takes
-a `Span parent`.  The result of `parent.child()` will be used for the new span.
-`Span.child()` simply returns a span that is a child of `this`.
-
-###Span Receivers
-In order to use the tracing information consisting of spans,
-you need an implementation of `SpanReceiver` interface which collects spans
-and typically writes it to files or databases or collector services.
-The `SpanReceiver` implementation must provide a `receiveSpan` method which
-is called from `Trace.deliver` method.
-You do not need to explicitly call `Trace.deliver`
-because it is internally called by the implementation of `Span`.
-
-````java
-    public interface SpanReceiver extends Closeable {
-      public void receiveSpan(Span span);
-    }
-````
-
-HTrace comes with several standard span receivers, such as
-`LocalFileSpanReceiver`.  An application can use the `SpanReceiverBuilder` to
-create a particular type of standard `SpanReceiver` based on the current
-configuration.  Once a SpanReceiver has been created, it should be registered
-with the HTrace framework by calling `Trace.addReceiver`.
-
-````java
-    HTraceConfiguration hconf = createMyHTraceConfiguration();
-    SpanReceiverBuilder builder = new SpanReceiverBuilder(hconf);
-    SpanReceiver spanReceiver = builder.build();
-    if (spanReceiver != null) {
-      Trace.addReceiver(spanReceiver);
-    }
-````
-
-####Zipkin
+#### htrace-zipkin
 htrace-zipkin provides the `SpanReceiver` implementation
 which sends spans to [Zipkin](https://github.com/twitter/zipkin) collector.
 You can build the uber-jar (htrace-zipkin-*-jar-withdependency.jar) for manual
@@ -230,34 +144,16 @@ htrace-core and its dependencies.
     $ cd htrace-zipkin
     $ mvn compile assembly:single
 
-####HBase Receiver
+#### htrace-hbase
 See htrace-hbase for an Span Receiver implementation that writes HBase.
-Also bundled is a simple Span Viewer.
 
-Testing Information
+Generating test spans
 -------------------------------
-
 The test that creates a sample trace (TestHTrace) takes a command line
 argument telling it where to write span information. Run
 `mvn test -DargLine="-DspanFile=FILE\_PATH"` to write span
 information to FILE_PATH. If no file is specified, span information
-will be written to standard out. If span information is written to a
-file, you can use the included graphDrawer python script in tools/
-to create a simple visualization of the trace. Or you could write
-some javascript to make a better visualization, and send a pull
-request if you do :).
-
-Migrating to Apache HTrace (incubating) from org.htrace
--------------------------------
-First, the package has changed from org.htrace to org.apache.htrace.
-
-Also, here are some notes on what had to be done migrating:
-
-HTRACE-1 changes the SpanReceiver interface. HBase was instantiating
-SpanReceivers itself and calling .configure(HTraceConfiguration) on each
-one. The expectation now is that SpanReceiver implementations
-provide a constructor that takes a single parameter of HTraceConfiguration.
-
+will be written to standard out.
 
 Publishing to Maven Central
 -------------------------------



[2/2] incubator-htrace git commit: HTRACE-214. De-globalize Tracer.java (cmccabe)

Posted by cm...@apache.org.
HTRACE-214. De-globalize Tracer.java (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/7997d208
Tree: http://git-wip-us.apache.org/repos/asf/incubator-htrace/tree/7997d208
Diff: http://git-wip-us.apache.org/repos/asf/incubator-htrace/diff/7997d208

Branch: refs/heads/master
Commit: 7997d208989b5e5bf6734da64ec9249c0226c21f
Parents: 0f873fd
Author: Masatake Iwasaki <iw...@apache.org>
Authored: Fri Aug 21 16:00:36 2015 +0900
Committer: Colin P. Mccabe <cm...@apache.org>
Committed: Mon Aug 24 19:15:35 2015 -0700

----------------------------------------------------------------------
 .../org/apache/htrace/core/AlwaysSampler.java   |   3 +-
 .../org/apache/htrace/core/CountSampler.java    |   2 +-
 .../htrace/core/LocalFileSpanReceiver.java      |  20 +-
 .../java/org/apache/htrace/core/MilliSpan.java  |   1 -
 .../org/apache/htrace/core/NeverSampler.java    |   4 +-
 .../java/org/apache/htrace/core/NullScope.java  |  41 +-
 .../apache/htrace/core/POJOSpanReceiver.java    |   2 +-
 .../apache/htrace/core/ProbabilitySampler.java  |   7 +-
 .../java/org/apache/htrace/core/Sampler.java    |  96 +++-
 .../org/apache/htrace/core/SamplerBuilder.java  |  91 ----
 .../main/java/org/apache/htrace/core/Span.java  |   9 +-
 .../java/org/apache/htrace/core/SpanId.java     |   3 -
 .../org/apache/htrace/core/SpanReceiver.java    | 137 ++++-
 .../apache/htrace/core/SpanReceiverBuilder.java | 138 -----
 .../htrace/core/StandardOutSpanReceiver.java    |   2 +-
 .../main/java/org/apache/htrace/core/Trace.java | 219 --------
 .../org/apache/htrace/core/TraceCallable.java   |  39 +-
 .../htrace/core/TraceExecutorService.java       |  19 +-
 .../java/org/apache/htrace/core/TraceProxy.java |  58 --
 .../org/apache/htrace/core/TraceRunnable.java   |  40 +-
 .../java/org/apache/htrace/core/TraceScope.java | 115 ++--
 .../java/org/apache/htrace/core/Tracer.java     | 528 +++++++++++++++++--
 .../org/apache/htrace/core/TracerBuilder.java   | 144 +++++
 .../java/org/apache/htrace/core/TracerId.java   |  44 +-
 .../java/org/apache/htrace/core/TracerPool.java | 283 ++++++++++
 .../org/apache/htrace/core/TestBadClient.java   | 171 ++++--
 .../java/org/apache/htrace/core/TestHTrace.java | 152 +++---
 .../htrace/core/TestLocalFileSpanReceiver.java  |  35 +-
 .../org/apache/htrace/core/TestNullScope.java   |  25 +-
 .../org/apache/htrace/core/TestSampler.java     |  89 +++-
 .../htrace/core/TestSpanReceiverBuilder.java    | 156 +++---
 .../org/apache/htrace/core/TestTracerId.java    |  21 +-
 .../org/apache/htrace/core/TraceCreator.java    |  58 +-
 .../apache/htrace/impl/FlumeSpanReceiver.java   |   7 +-
 .../htrace/impl/TestFlumeSpanReceiver.java      |  68 +--
 .../apache/htrace/HBaseSpanReceiverHost.java    | 107 ----
 .../apache/htrace/impl/HBaseSpanReceiver.java   |  35 +-
 .../org/apache/htrace/impl/HBaseTestUtil.java   |   4 +-
 .../htrace/impl/TestHBaseSpanReceiver.java      |  40 +-
 .../apache/htrace/impl/HTracedRESTReceiver.java |  10 +-
 .../htrace/impl/TestHTracedRESTReceiver.java    |   2 +
 .../apache/htrace/impl/ZipkinSpanReceiver.java  |   9 +-
 .../htrace/TestHTraceSpanToZipkinSpan.java      |  10 +-
 src/main/site/markdown/index.md                 | 186 ++-----
 44 files changed, 1845 insertions(+), 1385 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/AlwaysSampler.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/AlwaysSampler.java b/htrace-core/src/main/java/org/apache/htrace/core/AlwaysSampler.java
index a9259bd..8d5a296 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/AlwaysSampler.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/AlwaysSampler.java
@@ -19,8 +19,7 @@ package org.apache.htrace.core;
 /**
  * A Sampler that always returns true.
  */
-public final class AlwaysSampler implements Sampler {
-
+public final class AlwaysSampler extends Sampler {
   public static final AlwaysSampler INSTANCE = new AlwaysSampler(null);
 
   public AlwaysSampler(HTraceConfiguration conf) {

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/CountSampler.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/CountSampler.java b/htrace-core/src/main/java/org/apache/htrace/core/CountSampler.java
index 10d5c98..5a838c7 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/CountSampler.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/CountSampler.java
@@ -22,7 +22,7 @@ import java.util.concurrent.ThreadLocalRandom;
  * Sampler that returns true every N calls. Specify the frequency interval by configuring a
  * {@code long} value for {@link #SAMPLER_FREQUENCY_CONF_KEY}.
  */
-public class CountSampler implements Sampler {
+public class CountSampler extends Sampler {
   public final static String SAMPLER_FREQUENCY_CONF_KEY = "sampler.frequency";
 
   final long frequency;

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/LocalFileSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/LocalFileSpanReceiver.java b/htrace-core/src/main/java/org/apache/htrace/core/LocalFileSpanReceiver.java
index 0aed846..69a43b1 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/LocalFileSpanReceiver.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/LocalFileSpanReceiver.java
@@ -41,10 +41,10 @@ import java.util.concurrent.locks.ReentrantLock;
 /**
  * Writes the spans it receives to a local file.
  */
-public class LocalFileSpanReceiver implements SpanReceiver {
+public class LocalFileSpanReceiver extends SpanReceiver {
   private static final Log LOG = LogFactory.getLog(LocalFileSpanReceiver.class);
-  public static final String PATH_KEY = "local-file-span-receiver.path";
-  public static final String CAPACITY_KEY = "local-file-span-receiver.capacity";
+  public static final String PATH_KEY = "local.file.span.receiver.path";
+  public static final String CAPACITY_KEY = "local.file.span.receiver.capacity";
   public static final int CAPACITY_DEFAULT = 5000;
   private static ObjectWriter JSON_WRITER = new ObjectMapper().writer();
   private final String path;
@@ -56,7 +56,6 @@ public class LocalFileSpanReceiver implements SpanReceiver {
   private final FileOutputStream stream;
   private final FileChannel channel;
   private final ReentrantLock channelLock = new ReentrantLock();
-  private final TracerId tracerId;
 
   public LocalFileSpanReceiver(HTraceConfiguration conf) {
     int capacity = conf.getInt(CAPACITY_KEY, CAPACITY_DEFAULT);
@@ -64,9 +63,11 @@ public class LocalFileSpanReceiver implements SpanReceiver {
       throw new IllegalArgumentException(CAPACITY_KEY + " must not be " +
           "less than 1.");
     }
-    this.path = conf.get(PATH_KEY);
-    if (path == null || path.isEmpty()) {
-      throw new IllegalArgumentException("must configure " + PATH_KEY);
+    String pathStr = conf.get(PATH_KEY);
+    if (pathStr == null || pathStr.isEmpty()) {
+      path = getUniqueLocalTraceFileName();
+    } else {
+      path = pathStr;
     }
     boolean success = false;
     try {
@@ -91,7 +92,6 @@ public class LocalFileSpanReceiver implements SpanReceiver {
       LOG.debug("Created new LocalFileSpanReceiver with path = " + path +
                 ", capacity = " + capacity);
     }
-    this.tracerId = new TracerId(conf);
   }
 
   /**
@@ -134,10 +134,6 @@ public class LocalFileSpanReceiver implements SpanReceiver {
 
   @Override
   public void receiveSpan(Span span) {
-    if (span.getTracerId().isEmpty()) {
-      span.setTracerId(tracerId.get());
-    }
-
     // Serialize the span data into a byte[].  Note that we're not holding the
     // lock here, to improve concurrency.
     byte jsonBuf[] = null;

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/MilliSpan.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/MilliSpan.java b/htrace-core/src/main/java/org/apache/htrace/core/MilliSpan.java
index afef809..5dd6bdb 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/MilliSpan.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/MilliSpan.java
@@ -170,7 +170,6 @@ public class MilliSpan implements Span {
         throw new IllegalStateException("Span for " + description
             + " has not been started");
       end = System.currentTimeMillis();
-      Tracer.getInstance().deliver(this);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/NeverSampler.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/NeverSampler.java b/htrace-core/src/main/java/org/apache/htrace/core/NeverSampler.java
index 65f6087..60cc7d2 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/NeverSampler.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/NeverSampler.java
@@ -19,8 +19,7 @@ package org.apache.htrace.core;
 /**
  * A Sampler that never returns true.
  */
-public final class NeverSampler implements Sampler {
-
+public final class NeverSampler extends Sampler {
   public static final NeverSampler INSTANCE = new NeverSampler(null);
 
   public NeverSampler(HTraceConfiguration conf) {
@@ -30,5 +29,4 @@ public final class NeverSampler implements Sampler {
   public boolean next() {
     return false;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/NullScope.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/NullScope.java b/htrace-core/src/main/java/org/apache/htrace/core/NullScope.java
index e7964cf..fe76e46 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/NullScope.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/NullScope.java
@@ -17,28 +17,53 @@
 package org.apache.htrace.core;
 
 /**
- * Singleton instance representing an empty {@link TraceScope}.
+ * An empty {@link TraceScope}.
  */
-public final class NullScope extends TraceScope {
+class NullScope extends TraceScope {
+  NullScope(Tracer tracer) {
+    super(tracer, null, null);
+  }
 
-  public static final TraceScope INSTANCE = new NullScope();
+  @Override
+  public SpanId getSpanId() {
+    return SpanId.INVALID;
+  }
 
-  private NullScope() {
-    super(null, null);
+  @Override
+  public void detach() {
+    if (detached) {
+      Tracer.throwClientError("Can't detach this TraceScope  because " +
+          "it is already detached.");
+    }
+    detached = true;
   }
 
   @Override
-  public Span detach() {
-    return null;
+  public void reattach() {
+    if (!detached) {
+      Tracer.throwClientError("Can't reattach this TraceScope  because " +
+          "it is not detached.");
+    }
+    detached = false;
   }
 
   @Override
   public void close() {
-    return;
+    tracer.popNullScope();
   }
 
   @Override
   public String toString() {
     return "NullScope";
   }
+
+  @Override
+  public void addKVAnnotation(String key, String value) {
+    // do nothing
+  }
+
+  @Override
+  public void addTimelineAnnotation(String msg) {
+    // do nothing
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/POJOSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/POJOSpanReceiver.java b/htrace-core/src/main/java/org/apache/htrace/core/POJOSpanReceiver.java
index be782ba..34322fa 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/POJOSpanReceiver.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/POJOSpanReceiver.java
@@ -24,7 +24,7 @@ import java.util.HashSet;
  * SpanReceiver for testing only that just collects the Span objects it
  * receives. The spans it receives can be accessed with getSpans();
  */
-public class POJOSpanReceiver implements SpanReceiver {
+public class POJOSpanReceiver extends SpanReceiver {
   private final Collection<Span> spans;
 
   public POJOSpanReceiver(HTraceConfiguration conf) {

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/ProbabilitySampler.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/ProbabilitySampler.java b/htrace-core/src/main/java/org/apache/htrace/core/ProbabilitySampler.java
index 5bb0042..c0bb16c 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/ProbabilitySampler.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/ProbabilitySampler.java
@@ -16,17 +16,16 @@
  */
 package org.apache.htrace.core;
 
+import java.util.concurrent.ThreadLocalRandom;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import java.util.Random;
-import java.util.concurrent.ThreadLocalRandom;
-
 /**
  * Sampler that returns true a certain percentage of the time. Specify the frequency interval by
  * configuring a {@code double} value for {@link #SAMPLER_FRACTION_CONF_KEY}.
  */
-public class ProbabilitySampler implements Sampler {
+public class ProbabilitySampler extends Sampler {
   private static final Log LOG = LogFactory.getLog(ProbabilitySampler.class);
   public final double threshold;
   public final static String SAMPLER_FRACTION_CONF_KEY = "sampler.fraction";

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/Sampler.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/Sampler.java b/htrace-core/src/main/java/org/apache/htrace/core/Sampler.java
index 91843f5..af0165c 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/Sampler.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/Sampler.java
@@ -16,6 +16,11 @@
  */
 package org.apache.htrace.core;
 
+import java.lang.reflect.Constructor;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 /**
  * Extremely simple callback to determine the frequency that an action should be
  * performed.
@@ -31,9 +36,96 @@ package org.apache.htrace.core;
  * </pre>
  * This would trace 50% of all gets, 75% of all puts and would not trace any other requests.
  */
-public interface Sampler {
+public abstract class Sampler {
+  /**
+   * A {@link Sampler} builder. It takes a {@link Sampler} class name and
+   * constructs an instance of that class, with the provided configuration.
+   */
+  public static class Builder {
+    private static final Log LOG = LogFactory.getLog(Builder.class);
+
+    private final static String DEFAULT_PACKAGE = "org.apache.htrace.core";
+    private final HTraceConfiguration conf;
+    private String className;
+    private ClassLoader classLoader = Builder.class.getClassLoader();
+
+    public Builder(HTraceConfiguration conf) {
+      this.conf = conf;
+      reset();
+    }
+
+    public Builder reset() {
+      this.className = null;
+      return this;
+    }
+
+    public Builder className(String className) {
+      this.className = className;
+      return this;
+    }
+
+    public Builder classLoader(ClassLoader classLoader) {
+      this.classLoader = classLoader;
+      return this;
+    }
+
+    private void throwError(String errorStr) {
+      LOG.error(errorStr);
+      throw new RuntimeException(errorStr);
+    }
+
+    private void throwError(String errorStr, Throwable e) {
+      LOG.error(errorStr, e);
+      throw new RuntimeException(errorStr, e);
+    }
+
+    public Sampler build() {
+      Sampler sampler = newSampler();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Created new sampler of type " +
+            sampler.getClass().getName(), new Exception());
+      }
+      return sampler;
+    }
+
+    private Sampler newSampler() {
+      if (className == null || className.isEmpty()) {
+        throwError("No sampler class specified.");
+      }
+      String str = className;
+      if (!str.contains(".")) {
+        str = DEFAULT_PACKAGE + "." + str;
+      }
+      Class cls = null;
+      try {
+        cls = classLoader.loadClass(str);
+      } catch (ClassNotFoundException e) {
+        throwError("Cannot find Sampler class " + str);
+      }
+      Constructor<Sampler> ctor = null;
+      try {
+        ctor = cls.getConstructor(HTraceConfiguration.class);
+      } catch (NoSuchMethodException e) {
+        throwError("Cannot find a constructor for class " +
+            str + "which takes an HTraceConfiguration.");
+      }
+      Sampler sampler = null;
+      try {
+        LOG.debug("Creating new instance of " + str + "...");
+        sampler = ctor.newInstance(conf);
+      } catch (ReflectiveOperationException e) {
+        throwError("Reflection error when constructing " +
+            str + ".", e);
+      } catch (Throwable t) {
+        throwError("NewInstance error when constructing " +
+            str + ".", t);
+      }
+      return sampler;
+    }
+  }
+
   public static final Sampler ALWAYS = AlwaysSampler.INSTANCE;
   public static final Sampler NEVER = NeverSampler.INSTANCE;
 
-  public boolean next();
+  public abstract boolean next();
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/SamplerBuilder.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/SamplerBuilder.java b/htrace-core/src/main/java/org/apache/htrace/core/SamplerBuilder.java
deleted file mode 100644
index 5b53905..0000000
--- a/htrace-core/src/main/java/org/apache/htrace/core/SamplerBuilder.java
+++ /dev/null
@@ -1,91 +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.htrace.core;
-
-import java.lang.reflect.Constructor;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * A {@link Sampler} builder. It reads a {@link Sampler} class name from the provided
- * configuration using the {@link #SAMPLER_CONF_KEY} key. Unqualified class names
- * are interpreted as members of the {@code org.apache.htrace.impl} package. The {@link #build()}
- * method constructs an instance of that class, initialized with the same configuration.
- */
-public class SamplerBuilder {
-
-  // TODO: should follow the same API as SpanReceiverBuilder
-
-  public final static String SAMPLER_CONF_KEY = "sampler";
-  private final static String DEFAULT_PACKAGE = "org.apache.htrace.core";
-  private final static ClassLoader classLoader =
-      SamplerBuilder.class.getClassLoader();
-  private final HTraceConfiguration conf;
-  private static final Log LOG = LogFactory.getLog(SamplerBuilder.class);
-
-  public SamplerBuilder(HTraceConfiguration conf) {
-    this.conf = conf;
-  }
-
-  public Sampler build() {
-    Sampler sampler = newSampler();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Created new sampler of type " +
-          sampler.getClass().getName(), new Exception());
-    }
-    return sampler;
-  }
-
-  private Sampler newSampler() {
-    String str = conf.get(SAMPLER_CONF_KEY);
-    if (str == null || str.isEmpty()) {
-      return NeverSampler.INSTANCE;
-    }
-    if (!str.contains(".")) {
-      str = DEFAULT_PACKAGE + "." + str;
-    }
-    Class cls = null;
-    try {
-      cls = classLoader.loadClass(str);
-    } catch (ClassNotFoundException e) {
-      LOG.error("SamplerBuilder cannot find sampler class " + str +
-          ": falling back on NeverSampler.");
-      return NeverSampler.INSTANCE;
-    }
-    Constructor<Sampler> ctor = null;
-    try {
-      ctor = cls.getConstructor(HTraceConfiguration.class);
-    } catch (NoSuchMethodException e) {
-      LOG.error("SamplerBuilder cannot find a constructor for class " + str +
-          "which takes an HTraceConfiguration.  Falling back on " +
-          "NeverSampler.");
-      return NeverSampler.INSTANCE;
-    }
-    try {
-      return ctor.newInstance(conf);
-    } catch (ReflectiveOperationException e) {
-      LOG.error("SamplerBuilder reflection error when constructing " + str +
-          ".  Falling back on NeverSampler.", e);
-      return NeverSampler.INSTANCE;
-    } catch (Throwable e) {
-      LOG.error("SamplerBuilder constructor error when constructing " + str +
-          ".  Falling back on NeverSampler.", e);
-      return NeverSampler.INSTANCE;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/Span.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/Span.java b/htrace-core/src/main/java/org/apache/htrace/core/Span.java
index db1a961..4971983 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/Span.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/Span.java
@@ -16,15 +16,14 @@
  */
 package org.apache.htrace.core;
 
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.databind.JsonSerializer;
-import com.fasterxml.jackson.databind.SerializerProvider;
-import com.fasterxml.jackson.databind.annotation.JsonSerialize;
-
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 
 /**
  * Base interface for gathering and reporting statistics about a block of

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/SpanId.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/SpanId.java b/htrace-core/src/main/java/org/apache/htrace/core/SpanId.java
index e10f894..ed31ad3 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/SpanId.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/SpanId.java
@@ -16,10 +16,7 @@
  */
 package org.apache.htrace.core;
 
-import java.math.BigInteger;
-import java.lang.Void;
 import java.util.concurrent.ThreadLocalRandom;
-import java.util.Random;
 
 /**
  * Uniquely identifies an HTrace span.

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/SpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/SpanReceiver.java b/htrace-core/src/main/java/org/apache/htrace/core/SpanReceiver.java
index 5547c51..a955ddf 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/SpanReceiver.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/SpanReceiver.java
@@ -16,9 +16,12 @@
  */
 package org.apache.htrace.core;
 
-
 import java.io.Closeable;
+import java.lang.reflect.Constructor;
+import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 /**
  * The collector within a process that is the destination of Spans when a trace is running.
@@ -27,13 +30,135 @@ import java.io.Closeable;
  * <pre>
  * <code>public SpanReceiverImpl(HTraceConfiguration)</code>
  * </pre>
- * The helper class {@link org.apache.htrace.SpanReceiverBuilder} provides convenient factory
- * methods for creating {@code SpanReceiver} instances from configuration.
- * @see org.apache.htrace.SpanReceiverBuilder
  */
-public interface SpanReceiver extends Closeable {
+public abstract class SpanReceiver implements Closeable {
+  /**
+   * A {@link SpanReceiver} builder. It takes a {@link SpanReceiver} class name
+   * and constructs an instance of that class, with the provided configuration.
+   */
+  public static class Builder {
+    private static final Log LOG = LogFactory.getLog(Builder.class);
+
+    private final static String DEFAULT_PACKAGE = "org.apache.htrace.core";
+    private final HTraceConfiguration conf;
+    private boolean logErrors;
+    private String className;
+    private ClassLoader classLoader = Builder.class.getClassLoader();
+
+    public Builder(HTraceConfiguration conf) {
+      this.conf = conf;
+      reset();
+    }
+
+    /**
+     * Set this builder back to defaults.
+     *
+     * @return this instance.
+     */
+    public Builder reset() {
+      this.logErrors = true;
+      this.className = null;
+      return this;
+    }
+
+    public Builder className(final String className) {
+      this.className = className;
+      return this;
+    }
+
+    /**
+     * Configure whether we should log errors during build().
+     * @return This instance
+     */
+    public Builder logErrors(boolean logErrors) {
+      this.logErrors = logErrors;
+      return this;
+    }
+
+    public Builder classLoader(ClassLoader classLoader) {
+      this.classLoader = classLoader;
+      return this;
+    }
+
+    private void throwError(String errorStr) {
+      if (logErrors) {
+        LOG.error(errorStr);
+      }
+      throw new RuntimeException(errorStr);
+    }
+
+    private void throwError(String errorStr, Throwable e) {
+      if (logErrors) {
+        LOG.error(errorStr, e);
+      }
+      throw new RuntimeException(errorStr, e);
+    }
+
+    public SpanReceiver build() {
+      SpanReceiver spanReceiver = newSpanReceiver();
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("Created new span receiver of type " +
+                  spanReceiver.getClass().getName());
+      }
+      return spanReceiver;
+    }
+
+    private SpanReceiver newSpanReceiver() {
+      if ((className == null) || className.isEmpty()) {
+        throwError("No span receiver class specified.");
+      }
+      String str = className;
+      if (!str.contains(".")) {
+        str = DEFAULT_PACKAGE + "." + str;
+      }
+      Class cls = null;
+      try {
+        cls = classLoader.loadClass(str);
+      } catch (ClassNotFoundException e) {
+        throwError("Cannot find SpanReceiver class " + str);
+      }
+      Constructor<SpanReceiver> ctor = null;
+      try {
+        ctor = cls.getConstructor(HTraceConfiguration.class);
+      } catch (NoSuchMethodException e) {
+        throwError("Cannot find a constructor for class " +
+            str + "which takes an HTraceConfiguration.");
+      }
+      SpanReceiver receiver = null;
+      try {
+        LOG.debug("Creating new instance of " + str + "...");
+        receiver = ctor.newInstance(conf);
+      } catch (ReflectiveOperationException e) {
+        throwError("Reflection error when constructing " +
+            str + ".", e);
+      } catch (Throwable t) {
+        throwError("NewInstance error when constructing " +
+            str + ".", t);
+      }
+      return receiver;
+    }
+  }
+
+  /**
+   * An ID which uniquely identifies this SpanReceiver.
+   */
+  private final long id;
+
+  private static final AtomicLong HIGHEST_SPAN_RECEIVER_ID = new AtomicLong(0);
+
+  /**
+   * Get an ID uniquely identifying this SpanReceiver.
+   */
+  public final long getId() {
+    return id;
+  }
+
+  protected SpanReceiver() {
+    this.id = HIGHEST_SPAN_RECEIVER_ID.incrementAndGet();
+  }
+
   /**
    * Called when a Span is stopped and can now be stored.
    */
-  public void receiveSpan(Span span);
+  public abstract void receiveSpan(Span span);
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/SpanReceiverBuilder.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/SpanReceiverBuilder.java b/htrace-core/src/main/java/org/apache/htrace/core/SpanReceiverBuilder.java
deleted file mode 100644
index 3ab0b07..0000000
--- a/htrace-core/src/main/java/org/apache/htrace/core/SpanReceiverBuilder.java
+++ /dev/null
@@ -1,138 +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.htrace.core;
-
-import java.lang.reflect.Constructor;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-/**
- * A {@link SpanReceiver} builder. It reads a {@link SpanReceiver} class name from the provided
- * configuration using the {@link #SPAN_RECEIVER_CONF_KEY} key. Unqualified class names
- * are interpreted as members of the {@code org.apache.htrace.impl} package. The {@link #build()}
- * method constructs an instance of that class, initialized with the same configuration.
- */
-public class SpanReceiverBuilder {
-  private static final Log LOG = LogFactory.getLog(SpanReceiverBuilder.class);
-
-  public final static String SPAN_RECEIVER_CONF_KEY = "span.receiver";
-  private final static String DEFAULT_PACKAGE = "org.apache.htrace.core";
-  private final static ClassLoader classLoader =
-      SpanReceiverBuilder.class.getClassLoader();
-  private final HTraceConfiguration conf;
-  private boolean logErrors;
-  private String spanReceiverClass;
-
-  public SpanReceiverBuilder(HTraceConfiguration conf) {
-    this.conf = conf;
-    reset();
-  }
-
-  /**
-   * Set this builder back to defaults. Any previous calls to {@link #spanReceiverClass(String)}
-   * are overridden by the value provided by configuration.
-   * @return This instance
-   */
-  public SpanReceiverBuilder reset() {
-    this.logErrors = true;
-    this.spanReceiverClass = this.conf.get(SPAN_RECEIVER_CONF_KEY);
-    return this;
-  }
-
-  /**
-   * Override the {@code SpanReceiver} class name provided in configuration with a new value.
-   * @return This instance
-   */
-  public SpanReceiverBuilder spanReceiverClass(final String spanReceiverClass) {
-    this.spanReceiverClass = spanReceiverClass;
-    return this;
-  }
-
-  /**
-   * Configure whether we should log errors during build().
-   * @return This instance
-   */
-  public SpanReceiverBuilder logErrors(boolean logErrors) {
-    this.logErrors = logErrors;
-    return this;
-  }
-
-  private void logError(String errorStr) {
-    if (!logErrors) {
-      return;
-    }
-    LOG.error(errorStr);
-  }
-
-  private void logError(String errorStr, Throwable e) {
-    if (!logErrors) {
-      return;
-    }
-    LOG.error(errorStr, e);
-  }
-
-  public SpanReceiver build() {
-    SpanReceiver spanReceiver = newSpanReceiver();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Created new span receiver of type " +
-             ((spanReceiver == null) ? "(none)" :
-               spanReceiver.getClass().getName()));
-    }
-    return spanReceiver;
-  }
-
-  private SpanReceiver newSpanReceiver() {
-    if ((this.spanReceiverClass == null) ||
-        this.spanReceiverClass.isEmpty()) {
-      LOG.debug("No span receiver class specified.");
-      return null;
-    }
-    String str = spanReceiverClass;
-    if (!str.contains(".")) {
-      str = DEFAULT_PACKAGE + "." + str;
-    }
-    Class cls = null;
-    try {
-      cls = classLoader.loadClass(str);
-    } catch (ClassNotFoundException e) {
-      logError("SpanReceiverBuilder cannot find SpanReceiver class " + str +
-          ": disabling span receiver.");
-      return null;
-    }
-    Constructor<SpanReceiver> ctor = null;
-    try {
-      ctor = cls.getConstructor(HTraceConfiguration.class);
-    } catch (NoSuchMethodException e) {
-      logError("SpanReceiverBuilder cannot find a constructor for class " +
-          str + "which takes an HTraceConfiguration.  Disabling span " +
-          "receiver.");
-      return null;
-    }
-    try {
-      LOG.debug("Creating new instance of " + str + "...");
-      return ctor.newInstance(conf);
-    } catch (ReflectiveOperationException e) {
-      logError("SpanReceiverBuilder reflection error when constructing " + str +
-          ".  Disabling span receiver.", e);
-      return null;
-    } catch (Throwable e) {
-      logError("SpanReceiverBuilder constructor error when constructing " + str +
-          ".  Disabling span receiver.", e);
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/StandardOutSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/StandardOutSpanReceiver.java b/htrace-core/src/main/java/org/apache/htrace/core/StandardOutSpanReceiver.java
index b084046..f443ec6 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/StandardOutSpanReceiver.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/StandardOutSpanReceiver.java
@@ -24,7 +24,7 @@ import java.io.IOException;
 /**
  * Used for testing. Simply prints to standard out any spans it receives.
  */
-public class StandardOutSpanReceiver implements SpanReceiver {
+public class StandardOutSpanReceiver extends SpanReceiver {
   private static final Log LOG = LogFactory.getLog(StandardOutSpanReceiver.class);
 
   public StandardOutSpanReceiver(HTraceConfiguration conf) {

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/Trace.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/Trace.java b/htrace-core/src/main/java/org/apache/htrace/core/Trace.java
deleted file mode 100644
index 9b72afe..0000000
--- a/htrace-core/src/main/java/org/apache/htrace/core/Trace.java
+++ /dev/null
@@ -1,219 +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.htrace.core;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import java.util.concurrent.Callable;
-
-/**
- * The Trace class is the primary way to interact with the library.  It provides
- * methods to create and manipulate spans.
- *
- * A 'Span' represents a length of time.  It has many other attributes such as a
- * description, ID, and even potentially a set of key/value strings attached to
- * it.
- *
- * Each thread in your application has a single currently active currentSpan
- * associated with it.  When this is non-null, it represents the current
- * operation that the thread is doing.  Spans are NOT thread-safe, and must
- * never be used by multiple threads at once.  With care, it is possible to
- * safely pass a Span object between threads, but in most cases this is not
- * necessary.
- *
- * A 'TraceScope' can either be empty, or contain a Span.  TraceScope objects
- * implement the Java's Closeable interface.  Similar to file descriptors, they
- * must be closed after they are created.  When a TraceScope contains a Span,
- * this span is closed when the scope is closed.
- *
- * The 'startSpan' methods in this class do a few things:
- * <ul>
- *   <li>Create a new Span which has this thread's currentSpan as one of its parents.</li>
- *   <li>Set currentSpan to the new Span.</li>
- *   <li>Create a TraceSpan object to manage the new Span.</li>
- * </ul>
- *
- * Closing a TraceScope does a few things:
- * <ul>
- *   <li>It closes the span which the scope was managing.</li>
- *   <li>Set currentSpan to the previous currentSpan (which may be null).</li>
- * </ul>
- */
-public class Trace {
-  private static final Log LOG = LogFactory.getLog(Trace.class);
-
-  /**
-   * Creates a new trace scope.
-   *
-   * If this thread has a currently active trace span, the trace scope we create
-   * here will contain a new span descending from the currently active span.
-   * If there is no currently active trace span, the trace scope we create will
-   * be empty.
-   *
-   * @param description   The description field for the new span to create.
-   */
-  public static TraceScope startSpan(String description) {
-    return startSpan(description, NeverSampler.INSTANCE);
-  }
-
-  public static TraceScope startSpan(String description, SpanId parentId) {
-    if (parentId == null) {
-      return continueSpan(null);
-    }
-    Span newSpan = new MilliSpan.Builder().
-        begin(System.currentTimeMillis()).
-        end(0).
-        description(description).
-        spanId(parentId.newChildId()).
-        parents(new SpanId[] { parentId }).
-        build();
-    return continueSpan(newSpan);
-  }
-
-  /**
-   * Creates a new trace scope.
-   *
-   * If this thread has a currently active trace span, it must be the 'parent'
-   * span that you pass in here as a parameter.  The trace scope we create here
-   * will contain a new span which is a child of 'parent'.
-   *
-   * @param description   The description field for the new span to create.
-   */
-  public static TraceScope startSpan(String description, Span parent) {
-    if (parent == null) {
-      return startSpan(description);
-    }
-    Span currentSpan = currentSpan();
-    if ((currentSpan != null) && (currentSpan != parent)) {
-      Tracer.clientError("HTrace client error: thread " +
-          Thread.currentThread().getName() + " tried to start a new Span " +
-          "with parent " + parent.toString() + ", but there is already a " +
-          "currentSpan " + currentSpan);
-    }
-    return continueSpan(parent.child(description));
-  }
-
-  public static <T> TraceScope startSpan(String description, Sampler s) {
-    Span span = null;
-    if (isTracing() || s.next()) {
-      span = Tracer.getInstance().createNew(description);
-    }
-    return continueSpan(span);
-  }
-
-  /**
-   * Pick up an existing span from another thread.
-   */
-  public static TraceScope continueSpan(Span s) {
-    // Return an empty TraceScope that does nothing on close
-    if (s == null) return NullScope.INSTANCE;
-    return Tracer.getInstance().continueSpan(s);
-  }
-
-  /**
-   * Removes the given SpanReceiver from the list of SpanReceivers.
-   */
-  public static void removeReceiver(SpanReceiver rcvr) {
-    Tracer.getInstance().removeReceiver(rcvr);
-  }
-
-  /**
-   * Adds the given SpanReceiver to the current Tracer instance's list of
-   * SpanReceivers.
-   */
-  public static void addReceiver(SpanReceiver rcvr) {
-    Tracer.getInstance().addReceiver(rcvr);
-  }
-
-  /**
-   * Adds a data annotation to the current span if tracing is currently on.
-   */
-  public static void addKVAnnotation(String key, String value) {
-    Span s = currentSpan();
-    if (s != null) {
-      s.addKVAnnotation(key, value);
-    }
-  }
-
-  /**
-   * Annotate the current span with the given message.
-   */
-  public static void addTimelineAnnotation(String msg) {
-    Span s = currentSpan();
-    if (s != null) {
-      s.addTimelineAnnotation(msg);
-    }
-  }
-
-  /**
-   * Returns true if the current thread is a part of a trace, false otherwise.
-   */
-  public static boolean isTracing() {
-    return Tracer.getInstance().isTracing();
-  }
-
-  /**
-   * If we are tracing, return the current span, else null
-   *
-   * @return Span representing the current trace, or null if not tracing.
-   */
-  public static Span currentSpan() {
-    return Tracer.getInstance().currentSpan();
-  }
-
-  /**
-   * Wrap the callable in a TraceCallable, if tracing.
-   *
-   * @return The callable provided, wrapped if tracing, 'callable' if not.
-   */
-  public static <V> Callable<V> wrap(Callable<V> callable) {
-    if (isTracing()) {
-      return new TraceCallable<V>(Trace.currentSpan(), callable);
-    } else {
-      return callable;
-    }
-  }
-
-  /**
-   * Wrap the runnable in a TraceRunnable, if tracing
-   *
-   * @return The runnable provided, wrapped if tracing, 'runnable' if not.
-   */
-  public static Runnable wrap(Runnable runnable) {
-    if (isTracing()) {
-      return new TraceRunnable(Trace.currentSpan(), runnable);
-    } else {
-      return runnable;
-    }
-  }
-
-  /**
-   * Wrap the runnable in a TraceRunnable, if tracing
-   *
-   * @param description name of the span to be created.
-   * @param runnable The runnable that will have tracing info associated with it if tracing.
-   * @return The runnable provided, wrapped if tracing, 'runnable' if not.
-   */
-  public static Runnable wrap(String description, Runnable runnable) {
-    if (isTracing()) {
-      return new TraceRunnable(Trace.currentSpan(), runnable, description);
-    } else {
-      return runnable;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/TraceCallable.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/TraceCallable.java b/htrace-core/src/main/java/org/apache/htrace/core/TraceCallable.java
index 08bcace..a0fec17 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/TraceCallable.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/TraceCallable.java
@@ -22,44 +22,35 @@ import java.util.concurrent.Callable;
  * Wrap a Callable with a Span that survives a change in threads.
  */
 public class TraceCallable<V> implements Callable<V> {
+  private final Tracer tracer;
   private final Callable<V> impl;
-  private final Span parent;
+  private final TraceScope parent;
   private final String description;
 
-  public TraceCallable(Callable<V> impl) {
-    this(Trace.currentSpan(), impl);
-  }
-
-  public TraceCallable(Span parent, Callable<V> impl) {
-    this(parent, impl, null);
-  }
-
-  public TraceCallable(Span parent, Callable<V> impl, String description) {
+  TraceCallable(Tracer tracer, TraceScope parent, Callable<V> impl,
+      String description) {
+    this.tracer = tracer;
     this.impl = impl;
     this.parent = parent;
-    this.description = description;
+    if (description == null) {
+      this.description = Thread.currentThread().getName();
+    } else {
+      this.description = description;
+    }
   }
 
   @Override
   public V call() throws Exception {
-    if (parent != null) {
-      TraceScope chunk = Trace.startSpan(getDescription(), parent);
-
-      try {
-        return impl.call();
-      } finally {
-        chunk.close();
-      }
-    } else {
+    TraceScope chunk = tracer.newScope(description,
+        parent.getSpan().getSpanId());
+    try {
       return impl.call();
+    } finally {
+      chunk.close();
     }
   }
 
   public Callable<V> getImpl() {
     return impl;
   }
-
-  private String getDescription() {
-    return this.description == null ? Thread.currentThread().getName() : description;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/TraceExecutorService.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/TraceExecutorService.java b/htrace-core/src/main/java/org/apache/htrace/core/TraceExecutorService.java
index 8519d04..81e31ea 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/TraceExecutorService.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/TraceExecutorService.java
@@ -26,18 +26,21 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
-
 public class TraceExecutorService implements ExecutorService {
-
+  private final Tracer tracer;
+  private final String scopeName;
   private final ExecutorService impl;
 
-  public TraceExecutorService(ExecutorService impl) {
+  TraceExecutorService(Tracer tracer, String scopeName,
+                       ExecutorService impl) {
+    this.tracer = tracer;
+    this.scopeName = scopeName;
     this.impl = impl;
   }
 
   @Override
   public void execute(Runnable command) {
-    impl.execute(new TraceRunnable(command));
+    impl.execute(tracer.wrap(command, scopeName));
   }
 
   @Override
@@ -68,24 +71,24 @@ public class TraceExecutorService implements ExecutorService {
 
   @Override
   public <T> Future<T> submit(Callable<T> task) {
-    return impl.submit(new TraceCallable<T>(task));
+    return impl.submit(tracer.wrap(task, scopeName));
   }
 
   @Override
   public <T> Future<T> submit(Runnable task, T result) {
-    return impl.submit(new TraceRunnable(task), result);
+    return impl.submit(tracer.wrap(task, scopeName), result);
   }
 
   @Override
   public Future<?> submit(Runnable task) {
-    return impl.submit(new TraceRunnable(task));
+    return impl.submit(tracer.wrap(task, scopeName));
   }
 
   private <T> Collection<? extends Callable<T>> wrapCollection(
       Collection<? extends Callable<T>> tasks) {
     List<Callable<T>> result = new ArrayList<Callable<T>>();
     for (Callable<T> task : tasks) {
-      result.add(new TraceCallable<T>(task));
+      result.add(tracer.wrap(task, scopeName));
     }
     return result;
   }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/TraceProxy.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/TraceProxy.java b/htrace-core/src/main/java/org/apache/htrace/core/TraceProxy.java
deleted file mode 100644
index de9c980..0000000
--- a/htrace-core/src/main/java/org/apache/htrace/core/TraceProxy.java
+++ /dev/null
@@ -1,58 +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.htrace.core;
-
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-
-public class TraceProxy {
-  /**
-   * Returns an object that will trace all calls to itself.
-   */
-  public static <T> T trace(T instance) {
-    return trace(instance, Sampler.ALWAYS);
-  }
-
-  /**
-   * Returns an object that will trace all calls to itself.
-   */
-  @SuppressWarnings("unchecked")
-  public static <T, V> T trace(final T instance, final Sampler sampler) {
-    InvocationHandler handler = new InvocationHandler() {
-      @Override
-      public Object invoke(Object obj, Method method, Object[] args)
-          throws Throwable {
-        if (!sampler.next()) {
-          return method.invoke(instance, args);
-        }
-
-        TraceScope scope = Trace.startSpan(method.getName(), Sampler.ALWAYS);
-        try {
-          return method.invoke(instance, args);
-        } catch (Throwable ex) {
-          ex.printStackTrace();
-          throw ex;
-        } finally {
-          scope.close();
-        }
-      }
-    };
-    return (T) Proxy.newProxyInstance(instance.getClass().getClassLoader(),
-        instance.getClass().getInterfaces(), handler);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/TraceRunnable.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/TraceRunnable.java b/htrace-core/src/main/java/org/apache/htrace/core/TraceRunnable.java
index 6accea9..8f98708 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/TraceRunnable.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/TraceRunnable.java
@@ -20,44 +20,34 @@ package org.apache.htrace.core;
  * Wrap a Runnable with a Span that survives a change in threads.
  */
 public class TraceRunnable implements Runnable {
-
-  private final Span parent;
+  private final Tracer tracer;
+  private final TraceScope parent;
   private final Runnable runnable;
   private final String description;
 
-  public TraceRunnable(Runnable runnable) {
-    this(Trace.currentSpan(), runnable);
-  }
-
-  public TraceRunnable(Span parent, Runnable runnable) {
-    this(parent, runnable, null);
-  }
-
-  public TraceRunnable(Span parent, Runnable runnable, String description) {
+  public TraceRunnable(Tracer tracer, TraceScope parent,
+      Runnable runnable, String description) {
+    this.tracer = tracer;
     this.parent = parent;
     this.runnable = runnable;
-    this.description = description;
+    if (description == null) {
+      this.description = Thread.currentThread().getName();
+    } else {
+      this.description = description;
+    }
   }
 
   @Override
   public void run() {
-    if (parent != null) {
-      TraceScope chunk = Trace.startSpan(getDescription(), parent);
-
-      try {
-        runnable.run();
-      } finally {
-        chunk.close();
-      }
-    } else {
+    TraceScope chunk = tracer.newScope(description,
+        parent.getSpan().getSpanId());
+    try {
       runnable.run();
+    } finally {
+      chunk.close();
     }
   }
 
-  private String getDescription() {
-    return this.description == null ? Thread.currentThread().getName() : description;
-  }
-
   public Runnable getRunnable() {
     return runnable;
   }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/TraceScope.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/TraceScope.java b/htrace-core/src/main/java/org/apache/htrace/core/TraceScope.java
index f41e720..b04d785 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/TraceScope.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/TraceScope.java
@@ -18,6 +18,7 @@ package org.apache.htrace.core;
 
 import java.io.Closeable;
 import java.lang.Thread;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -25,75 +26,107 @@ public class TraceScope implements Closeable {
   private static final Log LOG = LogFactory.getLog(TraceScope.class);
 
   /**
-   * the span for this scope
+   * The tracer to use for this scope.
+   */
+  final Tracer tracer;
+
+  /**
+   * The trace span for this scope, or null if the scope is closed.
+   *
+   * If the scope is closed, it must also be detached.
    */
   private final Span span;
 
   /**
-   * the span that was "current" before this scope was entered
+   * The parent of this trace scope, or null if there is no parent.
    */
-  private final Span savedSpan;
+  private TraceScope parent;
 
-  private boolean detached = false;
+  /**
+   * True if this scope is detached.
+   */
+  boolean detached;
 
-  TraceScope(Span span, Span saved) {
+  TraceScope(Tracer tracer, Span span, TraceScope parent) {
+    this.tracer = tracer;
     this.span = span;
-    this.savedSpan = saved;
+    this.parent = parent;
+    this.detached = false;
   }
 
+  /**
+   * Returns the span which this scope is managing.
+   */
   public Span getSpan() {
     return span;
   }
 
   /**
-   * Remove this span as the current thread, but don't stop it yet or
-   * send it for collection. This is useful if the span object is then
-   * passed to another thread for use with Trace.continueTrace().
+   * Returns the span ID which this scope is managing.
+   */
+  public SpanId getSpanId() {
+    return span.getSpanId();
+  }
+
+  TraceScope getParent() {
+    return parent;
+  }
+
+  void setParent(TraceScope parent) {
+    this.parent = parent;
+  }
+
+  /**
+   * Detach this TraceScope from the current thread.
    *
-   * @return the same Span object
+   * It is OK to "leak" TraceScopes which have been detached.  They will not
+   * consume any resources other than a small amount of memory until they are
+   * garbage collected.  On the other hand, trace scopes which are still
+   * attached must never be leaked.
    */
-  public Span detach() {
+  public void detach() {
     if (detached) {
-      Tracer.clientError("Tried to detach trace span " + span + " but " +
-          "it has already been detached.");
+      Tracer.throwClientError("Can't detach this TraceScope  because " +
+          "it is already detached.");
     }
+    tracer.detachScope(this);
     detached = true;
+    parent = null;
+  }
 
-    Span cur = Tracer.getInstance().currentSpan();
-    if (cur != span) {
-      Tracer.clientError("Tried to detach trace span " + span + " but " +
-          "it is not the current span for the " +
-          Thread.currentThread().getName() + " thread.  You have " +
-          "probably forgotten to close or detach " + cur);
-    } else {
-      Tracer.getInstance().setCurrentSpan(savedSpan);
+  /**
+   * Attach this TraceScope to the current thread.
+   */
+  public void reattach() {
+    if (!detached) {
+      Tracer.throwClientError("Can't reattach this TraceScope  because " +
+          "it is not detached.");
     }
-    return span;
+    tracer.reattachScope(this);
+    detached = false;
   }
 
   /**
-   * Return true when {@link #detach()} has been called. Helpful when debugging
-   * multiple threads working on a single span.
+   * Close this TraceScope, ending the trace span it is managing.
    */
-  public boolean isDetached() {
-    return detached;
+  @Override
+  public void close() {
+    tracer.closeScope(this);
+
+  }
+
+  public void addKVAnnotation(String key, String value) {
+    span.addKVAnnotation(key, value);
+  }
+
+  public void addTimelineAnnotation(String msg) {
+    span.addTimelineAnnotation(msg);
   }
 
   @Override
-  public void close() {
-    if (detached) {
-      return;
-    }
-    detached = true;
-    Span cur = Tracer.getInstance().currentSpan();
-    if (cur != span) {
-      Tracer.clientError("Tried to close trace span " + span + " but " +
-          "it is not the current span for the " +
-          Thread.currentThread().getName() + " thread.  You have " +
-          "probably forgotten to close or detach " + cur);
-    } else {
-      span.stop();
-      Tracer.getInstance().setCurrentSpan(savedSpan);
-    }
+  public String toString() {
+    return "TraceScope(tracerId=" + tracer.getTracerId() +
+        ", span=" + span.toJson() +
+        ", detached=" + detached + ")";
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/Tracer.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/Tracer.java b/htrace-core/src/main/java/org/apache/htrace/core/Tracer.java
index b2ef6e6..6054a27 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/Tracer.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/Tracer.java
@@ -16,114 +16,526 @@
  */
 package org.apache.htrace.core;
 
+import java.io.Closeable;
+import java.lang.System;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.Arrays;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ThreadLocalRandom;
-
 /**
  * A Tracer provides the implementation for collecting and distributing Spans
  * within a process.
  */
-public class Tracer {
+public class Tracer implements Closeable {
   private static final Log LOG = LogFactory.getLog(Tracer.class);
 
-  static long nonZeroRandom64() {
-    long id;
-    Random random = ThreadLocalRandom.current();
-    do {
-      id = random.nextLong();
-    } while (id == 0);
-    return id;
-  }
+  /**
+   * The thread-specific context for this Tracer.
+   *
+   * This tracks the current number of trace scopes in a particular thread
+   * created by this tracer.  We use this to apply our samplers only for the
+   * "top-level" spans.
+   *
+   * Note that we can't put the TraceScope objects themselves in this context,
+   * since we need to be able to use TraceScopes created by other Tracers, and
+   * this context is per-Tracer.
+   */
+  private static class ThreadContext {
+    private long depth;
 
-  private final List<SpanReceiver> receivers = new CopyOnWriteArrayList<SpanReceiver>();
-  private static final ThreadLocal<Span> currentSpan = new ThreadLocal<Span>() {
+    ThreadContext() {
+      this.depth = 0;
+    }
+
+    boolean isTopLevel() {
+      return (depth == 0);
+    }
+
+    void pushScope() {
+      depth++;
+    }
+
+    TraceScope pushNewScope(Tracer tracer, Span span, TraceScope parentScope) {
+      TraceScope scope = new TraceScope(tracer, span, parentScope);
+      threadLocalScope.set(scope);
+      depth++;
+      return scope;
+    }
+
+    void popScope() {
+      if (depth <= 0) {
+        throwClientError("There were more trace scopes closed than " +
+            "were opened.");
+      }
+      depth--;
+    }
+  };
+
+  /**
+   * A subclass of ThreadLocal that starts off with a non-null initial value in
+   * each thread.
+   */
+  private static class ThreadLocalContext extends ThreadLocal<ThreadContext> {
     @Override
-    protected Span initialValue() {
-      return null;
+    protected ThreadContext initialValue() {
+      return new ThreadContext();
     }
   };
+
+  /**
+   * The current trace scope.  This is global, so it is shared amongst all
+   * libraries using HTrace.
+   */
+  final static ThreadLocal<TraceScope> threadLocalScope =
+      new ThreadLocal<TraceScope>();
+
+  /**
+   * An empty array of SpanId objects.  Can be used rather than constructing a
+   * new object whenever we need an empty array.
+   */
   private static final SpanId EMPTY_PARENT_ARRAY[] = new SpanId[0];
 
   /**
+   * The tracerId.
+   */
+  private final String tracerId;
+
+  /**
+   * The TracerPool which this Tracer belongs to.
+   *
+   * This gets set to null after the Tracer is closed in order to catch some
+   * use-after-close errors.  Note that we do not synchronize access on this
+   * field, since it only changes when the Tracer is closed, and the Tracer
+   * should not be used after that.
+   */
+  private TracerPool tracerPool;
+
+  /**
+   * The current thread-local context for this particualr Tracer.
+   */
+  private final ThreadLocalContext threadContext;
+
+  /**
+   * The NullScope instance for this Tracer.
+   */
+  private final NullScope nullScope;
+
+  /**
+   * The currently active Samplers.
+   *
+   * Arrays are immutable once set.  You must take the Tracer lock in order to
+   * set this to a new array.  If this is null, the Tracer is closed.
+   */
+  private volatile Sampler[] curSamplers;
+
+  /**
    * Log a client error, and throw an exception.
    *
    * @param str     The message to use in the log and the exception.
    */
-  static void clientError(String str) {
+  static void throwClientError(String str) {
     LOG.error(str);
     throw new RuntimeException(str);
   }
 
   /**
-   * Internal class for defered singleton idiom.
-   * <p/>
-   * https://en.wikipedia.org/wiki/Initialization_on_demand_holder_idiom
+   * If the current thread is tracing, this function returns the Tracer that is
+   * being used; otherwise, it returns null.
    */
-  private static class TracerHolder {
-    private static final Tracer INSTANCE = new Tracer();
+  public static Tracer curThreadTracer() {
+    TraceScope traceScope = threadLocalScope.get();
+    if (traceScope == null) {
+      return null;
+    }
+    return traceScope.tracer;
   }
 
-  public static Tracer getInstance() {
-    return TracerHolder.INSTANCE;
+  Tracer(String tracerId, TracerPool tracerPool, Sampler[] curSamplers) {
+    this.tracerId = tracerId;
+    this.tracerPool = tracerPool;
+    this.threadContext = new ThreadLocalContext();
+    this.nullScope = new NullScope(this);
+    this.curSamplers = curSamplers;
   }
 
-  protected Span createNew(String description) {
-    Span parent = currentSpan.get();
-    if (parent == null) {
-      return new MilliSpan.Builder().
-          begin(System.currentTimeMillis()).
-          end(0).
-          description(description).
-          parents(EMPTY_PARENT_ARRAY).
-          spanId(SpanId.fromRandom()).
-          build();
-    } else {
-      return parent.child(description);
+  public String getTracerId() {
+    return tracerId;
+  }
+
+  private TraceScope newScopeImpl(ThreadContext context, String description) {
+    Span span = new MilliSpan.Builder().
+        tracerId(tracerId).
+        begin(System.currentTimeMillis()).
+        description(description).
+        parents(EMPTY_PARENT_ARRAY).
+        spanId(SpanId.fromRandom()).
+        build();
+    return context.pushNewScope(this, span, null);
+  }
+
+  private TraceScope newScopeImpl(ThreadContext context, String description,
+        TraceScope parentScope) {
+    SpanId parentId = parentScope.getSpan().getSpanId();
+    Span span = new MilliSpan.Builder().
+        tracerId(tracerId).
+        begin(System.currentTimeMillis()).
+        description(description).
+        parents(new SpanId[] { parentId }).
+        spanId(parentId.newChildId()).
+        build();
+    return context.pushNewScope(this, span, parentScope);
+  }
+
+  private TraceScope newScopeImpl(ThreadContext context, String description,
+        SpanId parentId) {
+    Span span = new MilliSpan.Builder().
+        tracerId(tracerId).
+        begin(System.currentTimeMillis()).
+        description(description).
+        parents(new SpanId[] { parentId }).
+        spanId(parentId.newChildId()).
+        build();
+    return context.pushNewScope(this, span, null);
+  }
+
+  private TraceScope newScopeImpl(ThreadContext context, String description,
+        TraceScope parentScope, SpanId secondParentId) {
+    SpanId parentId = parentScope.getSpan().getSpanId();
+    Span span = new MilliSpan.Builder().
+        tracerId(tracerId).
+        begin(System.currentTimeMillis()).
+        description(description).
+        parents(new SpanId[] { parentId, secondParentId }).
+        spanId(parentId.newChildId()).
+        build();
+    return context.pushNewScope(this, span, parentScope);
+  }
+
+  /**
+   * Create a new trace scope.
+   *
+   * If there are no scopes above the current scope, we will apply our
+   * configured samplers.  Otherwise, we will create a span only if this thread
+   * is already tracing, or if the passed parentID was valid.
+   *
+   * @param description         The description of the new span to create.
+   * @param parentId            If this is a valid span ID, it will be added to
+   *                              the parents of the new span we create.
+   * @return                    The new trace scope.
+   */
+  public TraceScope newScope(String description, SpanId parentId) {
+    TraceScope parentScope = threadLocalScope.get();
+    ThreadContext context = threadContext.get();
+    if (parentScope != null) {
+      if (parentId.isValid() &&
+          (!parentId.equals(parentScope.getSpan().getSpanId()))) {
+        return newScopeImpl(context, description, parentScope, parentId);
+      } else {
+        return newScopeImpl(context, description, parentScope);
+      }
+    } else if (parentId.isValid()) {
+      return newScopeImpl(context, description, parentId);
+    }
+    if (!context.isTopLevel()) {
+      context.pushScope();
+      return nullScope;
     }
+    if (!sample()) {
+      context.pushScope();
+      return nullScope;
+    }
+    return newScopeImpl(context, description);
   }
 
-  protected boolean isTracing() {
-    return currentSpan.get() != null;
+  /**
+   * Create a new trace scope.
+   *
+   * If there are no scopes above the current scope, we will apply our
+   * configured samplers.  Otherwise, we will create a span only if this thread
+   * is already tracing.
+   */
+  public TraceScope newScope(String description) {
+    TraceScope parentScope = threadLocalScope.get();
+    ThreadContext context = threadContext.get();
+    if (parentScope != null) {
+      return newScopeImpl(context, description, parentScope);
+    }
+    if (!context.isTopLevel()) {
+      context.pushScope();
+      return nullScope;
+    }
+    if (!sample()) {
+      context.pushScope();
+      return nullScope;
+    }
+    return newScopeImpl(context, description);
   }
 
-  protected Span currentSpan() {
-    return currentSpan.get();
+  /**
+   * Return a null trace scope.
+   */
+  public TraceScope newNullScope() {
+    ThreadContext context = threadContext.get();
+    context.pushScope();
+    return nullScope;
   }
 
-  public void deliver(Span span) {
+  /**
+   * Wrap the callable in a TraceCallable, if tracing.
+   *
+   * @return The callable provided, wrapped if tracing, 'callable' if not.
+   */
+  public <V> Callable<V> wrap(Callable<V> callable, String description) {
+    TraceScope parentScope = threadLocalScope.get();
+    if (parentScope == null) {
+      return callable;
+    }
+    return new TraceCallable<V>(this, parentScope, callable, description);
+  }
+
+  /**
+   * Wrap the runnable in a TraceRunnable, if tracing
+   *
+   * @return The runnable provided, wrapped if tracing, 'runnable' if not.
+   */
+  public Runnable wrap(Runnable runnable, String description) {
+    TraceScope parentScope = threadLocalScope.get();
+    if (parentScope == null) {
+      return runnable;
+    }
+    return new TraceRunnable(this, parentScope, runnable, description);
+  }
+
+  public TraceExecutorService newTraceExecutorService(ExecutorService impl,
+                                                      String scopeName) {
+    return new TraceExecutorService(this, scopeName, impl);
+  }
+
+  public TracerPool getTracerPool() {
+    if (tracerPool == null) {
+      throwClientError(toString() + " is closed.");
+    }
+    return tracerPool;
+  }
+
+  /**
+   * Returns an object that will trace all calls to itself.
+   */
+  @SuppressWarnings("unchecked")
+  <T, V> T createProxy(final T instance) {
+    final Tracer tracer = this;
+    InvocationHandler handler = new InvocationHandler() {
+      @Override
+      public Object invoke(Object obj, Method method, Object[] args)
+          throws Throwable {
+        TraceScope scope = tracer.newScope(method.getName());
+        try {
+          return method.invoke(instance, args);
+        } catch (Throwable ex) {
+          ex.printStackTrace();
+          throw ex;
+        } finally {
+          scope.close();
+        }
+      }
+    };
+    return (T) Proxy.newProxyInstance(instance.getClass().getClassLoader(),
+        instance.getClass().getInterfaces(), handler);
+  }
+
+  /**
+   * Return true if we should create a new top-level span.
+   *
+   * We will create the span if any configured sampler returns true.
+   */
+  private boolean sample() {
+    Sampler[] samplers = curSamplers;
+    for (Sampler sampler : samplers) {
+      if (sampler.next()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Returns an array of all the current Samplers.
+   *
+   * Note that if the current Samplers change, those changes will not be
+   * reflected in this array.  In other words, this array may be stale.
+   */
+  public Sampler[] getSamplers() {
+    return curSamplers;
+  }
+
+  /**
+   * Add a new Sampler.
+   *
+   * @param sampler       The new sampler to add.
+   *                      You cannot add a particular Sampler object more
+   *                        than once.  You may add multiple Sampler objects
+   *                        of the same type, although this is not recommended.
+   *
+   * @return              True if the sampler was added; false if it already had
+   *                        been added earlier.
+   */
+  public synchronized boolean addSampler(Sampler sampler) {
+    if (tracerPool == null) {
+      throwClientError(toString() + " is closed.");
+    }
+    Sampler[] samplers = curSamplers;
+    int j = 0;
+    for (int i = 0; i < samplers.length; i++) {
+      if (samplers[i] == sampler) {
+        return false;
+      }
+    }
+    Sampler[] newSamplers =
+        Arrays.copyOf(samplers, samplers.length + 1);
+    newSamplers[samplers.length] = sampler;
+    curSamplers = newSamplers;
+    return true;
+  }
+
+  /**
+   * Remove a SpanReceiver.
+   *
+   * @param sampler       The sampler to remove.
+   */
+  public synchronized boolean removeSampler(Sampler sampler) {
+    if (tracerPool == null) {
+      throwClientError(toString() + " is closed.");
+    }
+    Sampler[] samplers = curSamplers;
+    int j = 0;
+    for (int i = 0; i < samplers.length; i++) {
+      if (samplers[i] == sampler) {
+        Sampler[] newSamplers = new Sampler[samplers.length - 1];
+        System.arraycopy(samplers, 0, newSamplers, 0, i);
+        System.arraycopy(samplers, i + 1, newSamplers, i,
+            samplers.length - i - 1);
+        curSamplers = newSamplers;
+        return true;
+      }
+    }
+    return false;
+  }
+
+  void detachScope(TraceScope scope) {
+    TraceScope curScope = threadLocalScope.get();
+    if (curScope != scope) {
+      throwClientError("Can't detach TraceScope for " +
+          scope.getSpan().toJson() + " because it is not the current " +
+          "TraceScope in thread " + Thread.currentThread().getName());
+    }
+    ThreadContext context = threadContext.get();
+    context.popScope();
+    threadLocalScope.set(scope.getParent());
+  }
+
+  void reattachScope(TraceScope scope) {
+    TraceScope parent = threadLocalScope.get();
+    Tracer.threadLocalScope.set(scope);
+    ThreadContext context = threadContext.get();
+    context.pushScope();
+    scope.setParent(parent);
+  }
+
+  void closeScope(TraceScope scope) {
+    TraceScope curScope = threadLocalScope.get();
+    if (curScope != scope) {
+      throwClientError("Can't close TraceScope for " +
+          scope.getSpan().toJson() + " because it is not the current " +
+          "TraceScope in thread " + Thread.currentThread().getName());
+    }
+    if (tracerPool == null) {
+      throwClientError(toString() + " is closed.");
+    }
+    SpanReceiver[] receivers = tracerPool.getReceivers();
+    if (receivers == null) {
+      throwClientError(toString() + " is closed.");
+    }
+    ThreadContext context = threadContext.get();
+    context.popScope();
+    threadLocalScope.set(scope.getParent());
+    scope.setParent(null);
+    Span span = scope.getSpan();
+    span.stop();
     for (SpanReceiver receiver : receivers) {
       receiver.receiveSpan(span);
     }
   }
 
-  protected void addReceiver(SpanReceiver receiver) {
-    receivers.add(receiver);
+  void popNullScope() {
+    TraceScope curScope = threadLocalScope.get();
+    if (curScope != null) {
+      throwClientError("Attempted to close an empty scope, but it was not " +
+          "the current thread scope in thread " +
+          Thread.currentThread().getName());
+    }
+    ThreadContext context = threadContext.get();
+    context.popScope();
   }
 
-  protected void removeReceiver(SpanReceiver receiver) {
-    receivers.remove(receiver);
+  public static Span getCurrentSpan() {
+    TraceScope curScope = threadLocalScope.get();
+    if (curScope == null) {
+      return null;
+    } else {
+      return curScope.getSpan();
+    }
+  }
+
+  public static SpanId getCurrentSpanId() {
+    TraceScope curScope = threadLocalScope.get();
+    if (curScope == null) {
+      return SpanId.INVALID;
+    } else {
+      return curScope.getSpan().getSpanId();
+    }
   }
 
-  protected Span setCurrentSpan(Span span) {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("setting current span " + span);
+  @Override
+  public synchronized void close() {
+    if (tracerPool == null) {
+      return;
     }
-    currentSpan.set(span);
-    return span;
+    curSamplers = new Sampler[0];
+    tracerPool.removeTracer(this);
   }
 
-  public TraceScope continueSpan(Span s) {
-    Span oldCurrent = currentSpan();
-    setCurrentSpan(s);
-    return new TraceScope(s, oldCurrent);
+  /**
+   * Get the hash code of a Tracer object.
+   *
+   * This hash code is based on object identity.
+   * This is used in TracerPool to create a hash table of Tracers.
+   */
+  @Override
+  public int hashCode() {
+    return System.identityHashCode(this);
   }
 
-  protected int numReceivers() {
-    return receivers.size();
+  /**
+   * Compare two tracer objects.
+   *
+   * Tracer objects are always compared by object equality.
+   * This is used in TracerPool to create a hash table of Tracers.
+   */
+  @Override
+  public boolean equals(Object other) {
+    return (this == other);
+  }
+
+  @Override
+  public String toString() {
+    return "Tracer(" + tracerId + ")";
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/TracerBuilder.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/TracerBuilder.java b/htrace-core/src/main/java/org/apache/htrace/core/TracerBuilder.java
new file mode 100644
index 0000000..0f12253
--- /dev/null
+++ b/htrace-core/src/main/java/org/apache/htrace/core/TracerBuilder.java
@@ -0,0 +1,144 @@
+/*
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.lang.reflect.Constructor;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Builds a new Tracer object.
+ */
+public class TracerBuilder {
+  public final static String SPAN_RECEIVER_CLASSES_KEY =
+      "span.receiver.classes";
+  public final static String SAMPLER_CLASSES_KEY =
+      "sampler.classes";
+
+  private static final Log LOG = LogFactory.getLog(TracerBuilder.class);
+
+  private String name;
+  private HTraceConfiguration conf = HTraceConfiguration.EMPTY;
+  private ClassLoader classLoader =
+      TracerBuilder.class.getClassLoader();
+  private TracerPool tracerPool = TracerPool.GLOBAL;
+
+  public TracerBuilder() {
+  }
+
+  public TracerBuilder name(String name) {
+    this.name = name;
+    return this;
+  }
+
+  public TracerBuilder conf(HTraceConfiguration conf) {
+    this.conf = conf;
+    return this;
+  }
+
+  public TracerBuilder tracerPool(TracerPool tracerPool) {
+    this.tracerPool = tracerPool;
+    return this;
+  }
+
+  private void loadSamplers(List<Sampler> samplers) {
+    String classNamesStr = conf.get(SAMPLER_CLASSES_KEY, "");
+    List<String> classNames = getClassNamesFromConf(classNamesStr);
+    StringBuilder bld = new StringBuilder();
+    String prefix = "";
+    for (String className : classNames) {
+      try {
+        Sampler sampler = new Sampler.Builder(conf).
+          className(className).
+          classLoader(classLoader).
+          build();
+        samplers.add(sampler);
+        bld.append(prefix).append(className);
+        prefix = ", ";
+      } catch (Throwable e) {
+        LOG.error("Failed to create SpanReceiver of type " + className, e);
+      }
+    }
+    String resultString = bld.toString();
+    if (resultString.isEmpty()) {
+      resultString = "no samplers";
+    }
+    LOG.info(SAMPLER_CLASSES_KEY + " = " + classNamesStr +
+        "; loaded " + resultString);
+  }
+
+  private void loadSpanReceivers() {
+    String classNamesStr = conf.get(SPAN_RECEIVER_CLASSES_KEY, "");
+    List<String> classNames = getClassNamesFromConf(classNamesStr);
+    StringBuilder bld = new StringBuilder();
+    String prefix = "";
+    for (String className : classNames) {
+      try {
+        tracerPool.loadReceiverType(className, conf, classLoader);
+        bld.append(prefix).append(className);
+        prefix = ", ";
+      } catch (Throwable e) {
+        LOG.error("Failed to create SpanReceiver of type " + className, e);
+      }
+    }
+    String resultString = bld.toString();
+    if (resultString.isEmpty()) {
+      resultString = "no span receivers";
+    }
+    LOG.info(SPAN_RECEIVER_CLASSES_KEY + " = " + classNamesStr +
+        "; loaded " + resultString);
+  }
+
+  /**
+   * Get a list of class names from the HTrace configuration.
+   * Entries which are empty will be removed.  Entries which lack a package will
+   * be given the default package.
+   *
+   * @param classNamesStr     A semicolon-separated string containing a list
+   *                            of class names.
+   * @return                  A list of class names.
+   */
+  private List<String> getClassNamesFromConf(String classNamesStr) {
+    String classNames[] = classNamesStr.split(";");
+    LinkedList<String> cleanedClassNames = new LinkedList<String>();
+    for (String className : classNames) {
+      String cleanedClassName = className.trim();
+      if (!cleanedClassName.isEmpty()) {
+        cleanedClassNames.add(cleanedClassName);
+      }
+    }
+    return cleanedClassNames;
+  }
+
+  public Tracer build() {
+    if (name == null) {
+      throw new RuntimeException("You must specify a name for this Tracer.");
+    }
+    LinkedList<SpanReceiver> spanReceivers = new LinkedList<SpanReceiver>();
+    LinkedList<Sampler> samplers = new LinkedList<Sampler>();
+    loadSamplers(samplers);
+    String tracerId = new TracerId(conf, name).get();
+    Tracer tracer = new Tracer(tracerId, tracerPool,
+        samplers.toArray(new Sampler[samplers.size()]));
+    tracerPool.addTracer(tracer);
+    loadSpanReceivers();
+    return tracer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/7997d208/htrace-core/src/main/java/org/apache/htrace/core/TracerId.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/core/TracerId.java b/htrace-core/src/main/java/org/apache/htrace/core/TracerId.java
index 7cdbd34..da482fe 100644
--- a/htrace-core/src/main/java/org/apache/htrace/core/TracerId.java
+++ b/htrace-core/src/main/java/org/apache/htrace/core/TracerId.java
@@ -16,9 +16,6 @@
  */
 package org.apache.htrace.core;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
@@ -30,6 +27,9 @@ import java.util.Enumeration;
 import java.util.Locale;
 import java.util.TreeSet;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 /**
  * The HTrace tracer ID.<p/>
  *
@@ -38,17 +38,19 @@ import java.util.TreeSet;
  * replace with the correct values at runtime.<p/>
  *
  * <ul>
- * <li>${ip}: will be replaced with an ip address.</li>
- * <li>${pname}: will be replaced the process name obtained from java.</li>
+ * <li>%{tname}: the tracer name supplied when creating the Tracer.</li>
+ * <li>%{pname}: the process name obtained from the JVM.</li>
+ * <li>%{ip}: will be replaced with an ip address.</li>
+ * <li>%{pid}: the numerical process ID from the operating system.</li>
  * </ul><p/>
  *
- * For example, the string "${pname}/${ip}" will be replaced with something
+ * For example, the string "%{pname}/%{ip}" will be replaced with something
  * like: DataNode/192.168.0.1, assuming that the process' name is DataNode
  * and its IP address is 192.168.0.1.<p/>
  *
- * Process ID strings can contain backslashes as escapes.
- * For example, "\a" will map to "a".  "\${ip}" will map to the literal
- * string "${ip}", not the IP address.  A backslash itself can be escaped by a
+ *  ID strings can contain backslashes as escapes.
+ * For example, "\a" will map to "a".  "\%{ip}" will map to the literal
+ * string "%{ip}", not the IP address.  A backslash itself can be escaped by a
  * preceding backslash.
  */
 public final class TracerId {
@@ -57,16 +59,20 @@ public final class TracerId {
   /**
    * The configuration key to use for process id
    */
-  public static final String TRACER_ID_KEY = "process.id";
+  public static final String TRACER_ID_KEY = "tracer.id";
 
   /**
-   * The default process ID to use if no other ID is configured.
+   * The default tracer ID to use if no other ID is configured.
    */
-  private static final String DEFAULT_TRACER_ID = "${pname}/${ip}";
+  private static final String DEFAULT_TRACER_ID = "%{tname}/%{ip}";
+
+  private final String tracerName;
 
   private final String tracerId;
 
-  TracerId(String fmt) {
+  public TracerId(HTraceConfiguration conf, String tracerName) {
+    this.tracerName = tracerName;
+    String fmt = conf.get(TRACER_ID_KEY, DEFAULT_TRACER_ID);
     StringBuilder bld = new StringBuilder();
     StringBuilder varBld = null;
     boolean escaping = false;
@@ -81,7 +87,7 @@ public final class TracerId {
       }
       switch (varSeen) {
         case 0:
-          if (c == '$') {
+          if (c == '%') {
             if (!escaping) {
               varSeen = 1;
               continue;
@@ -101,7 +107,7 @@ public final class TracerId {
           }
           escaping = false;
           varSeen = 0;
-          bld.append("$").append(c);
+          bld.append("%").append(c);
           break;
         default:
           if (c == '}') {
@@ -130,12 +136,10 @@ public final class TracerId {
     }
   }
 
-  public TracerId(HTraceConfiguration conf) {
-    this(conf.get(TRACER_ID_KEY, DEFAULT_TRACER_ID));
-  }
-
   private String processShellVar(String var) {
-    if (var.equals("pname")) {
+    if (var.equals("tname")) {
+      return tracerName;
+    } else if (var.equals("pname")) {
       return getProcessName();
     } else if (var.equals("ip")) {
       return getBestIpString();