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 2014/12/10 21:19:05 UTC

incubator-htrace git commit: HTRACE-1. HTrace should have a factory method for creating SpanReceiver objects (cmccabe)

Repository: incubator-htrace
Updated Branches:
  refs/heads/master b1456caf5 -> f43a0ccbf


HTRACE-1. HTrace should have a factory method for creating SpanReceiver objects (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/f43a0ccb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-htrace/tree/f43a0ccb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-htrace/diff/f43a0ccb

Branch: refs/heads/master
Commit: f43a0ccbffc80afbe15c816c2a6390a3a11cc5a8
Parents: b1456ca
Author: Colin P. Mccabe <cm...@apache.org>
Authored: Tue Dec 9 18:45:54 2014 -0800
Committer: Colin P. Mccabe <cm...@apache.org>
Committed: Wed Dec 10 12:18:23 2014 -0800

----------------------------------------------------------------------
 .../org/apache/htrace/HTraceConfiguration.java  |  14 +++
 .../java/org/apache/htrace/SpanReceiver.java    |   8 --
 .../org/apache/htrace/SpanReceiverFactory.java  |  95 ++++++++++++++
 .../htrace/impl/LocalFileSpanReceiver.java      |   7 +-
 .../apache/htrace/impl/POJOSpanReceiver.java    |   9 +-
 .../htrace/impl/StandardOutSpanReceiver.java    |   4 +-
 .../test/java/org/apache/htrace/TestHTrace.java |   8 +-
 .../apache/htrace/TestSpanReceiverFactory.java  | 123 +++++++++++++++++++
 .../apache/htrace/impl/HBaseSpanReceiver.java   |   6 +-
 .../apache/htrace/impl/ZipkinSpanReceiver.java  |   6 +-
 .../htrace/TestHTraceSpanToZipkinSpan.java      |   2 +-
 11 files changed, 245 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/htrace-core/src/main/java/org/apache/htrace/HTraceConfiguration.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/HTraceConfiguration.java b/htrace-core/src/main/java/org/apache/htrace/HTraceConfiguration.java
index ecef911..a5f244b 100644
--- a/htrace-core/src/main/java/org/apache/htrace/HTraceConfiguration.java
+++ b/htrace-core/src/main/java/org/apache/htrace/HTraceConfiguration.java
@@ -16,6 +16,7 @@
  */
 package org.apache.htrace;
 
+import com.google.common.collect.ImmutableMap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -30,6 +31,19 @@ public abstract class HTraceConfiguration {
 
   private static final Log LOG = LogFactory.getLog(HTraceConfiguration.class);
 
+  private static final Map<String, String> EMPTY_MAP = new HashMap<String, String>(1);
+
+  /**
+   * An empty HTrace configuration.
+   */
+  public static final HTraceConfiguration EMPTY = fromMap(EMPTY_MAP);
+
+  /**
+   * Create an HTrace configuration from a map.
+   *
+   * @param conf    The map to create the configuration from.
+   * @return        The new configuration.
+   */
   public static HTraceConfiguration fromMap(Map<String, String> conf) {
     return new MapConf(conf);
   }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/htrace-core/src/main/java/org/apache/htrace/SpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/SpanReceiver.java b/htrace-core/src/main/java/org/apache/htrace/SpanReceiver.java
index aad549a..22e8426 100644
--- a/htrace-core/src/main/java/org/apache/htrace/SpanReceiver.java
+++ b/htrace-core/src/main/java/org/apache/htrace/SpanReceiver.java
@@ -25,18 +25,10 @@ import java.io.Closeable;
  * is running.
  */
 public interface SpanReceiver extends Closeable {
-
-  /**
-   * Hosts of SpanReceivers should call this method to provide
-   * configuration to SpanReceivers after creating them.
-   */
-  public void configure(HTraceConfiguration conf);
-
   /**
    * Called when a Span is stopped and can now be stored.
    *
    * @param span
    */
   public void receiveSpan(Span span);
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/htrace-core/src/main/java/org/apache/htrace/SpanReceiverFactory.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/SpanReceiverFactory.java b/htrace-core/src/main/java/org/apache/htrace/SpanReceiverFactory.java
new file mode 100644
index 0000000..4d16ffb
--- /dev/null
+++ b/htrace-core/src/main/java/org/apache/htrace/SpanReceiverFactory.java
@@ -0,0 +1,95 @@
+/*
+ * 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.lang.reflect.Constructor;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class SpanReceiverFactory {
+  static final Log LOG = LogFactory.getLog(SpanReceiverFactory.class);
+
+  public final static String SPAN_RECEIVER_CONF_KEY = "span.receiver";
+  private final static ClassLoader classLoader =
+      SpanReceiverFactory.class.getClassLoader();
+  private final HTraceConfiguration conf;
+  private boolean logErrors = true;
+
+  public SpanReceiverFactory(HTraceConfiguration conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Configure whether we should log errors during build().
+   */
+  public SpanReceiverFactory 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() {
+    String str = conf.get(SPAN_RECEIVER_CONF_KEY);
+    if ((str == null) || str.isEmpty()) {
+      return null;
+    }
+    if (!str.contains(".")) {
+      str = "org.apache.htrace.impl." + str;
+    }
+    Class cls = null;
+    try {
+      cls = classLoader.loadClass(str);
+    } catch (ClassNotFoundException e) {
+      logError("SpanReceiverFactory cannot find SpanReceiver class " + str +
+          ": disabling span receiver.");
+      return null;
+    }
+    Constructor<SpanReceiver> ctor = null;
+    try {
+      ctor = cls.getConstructor(HTraceConfiguration.class);
+    } catch (NoSuchMethodException e) {
+      logError("SpanReceiverFactory cannot find a constructor for class " +
+          str + "which takes an HTraceConfiguration.  Disabling span " +
+          "receiver.");
+      return null;
+    }
+    try {
+      return ctor.newInstance(conf);
+    } catch (ReflectiveOperationException e) {
+      logError("SpanReceiverFactory reflection error when constructing " + str +
+          ".  Disabling span receiver.", e);
+      return null;
+    } catch (Throwable e) {
+      logError("SpanReceiverFactory constructor error when constructing " + str +
+          ".  Disabling span receiver.", e);
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/htrace-core/src/main/java/org/apache/htrace/impl/LocalFileSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/impl/LocalFileSpanReceiver.java b/htrace-core/src/main/java/org/apache/htrace/impl/LocalFileSpanReceiver.java
index 6b4be80..627b758 100644
--- a/htrace-core/src/main/java/org/apache/htrace/impl/LocalFileSpanReceiver.java
+++ b/htrace-core/src/main/java/org/apache/htrace/impl/LocalFileSpanReceiver.java
@@ -52,12 +52,7 @@ public class LocalFileSpanReceiver implements SpanReceiver {
   private ExecutorService executor;
   private long executorTerminationTimeoutDuration;
 
-  public LocalFileSpanReceiver() {
-  }
-
-
-  @Override
-  public void configure(HTraceConfiguration conf) {
+  public LocalFileSpanReceiver(HTraceConfiguration conf) {
     this.executorTerminationTimeoutDuration = EXECUTOR_TERMINATION_TIMEOUT_DURATION_DEFAULT;
     int capacity = conf.getInt(CAPACITY_KEY, CAPACITY_DEFAULT);
     this.file = conf.get(PATH_KEY);

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/htrace-core/src/main/java/org/apache/htrace/impl/POJOSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/impl/POJOSpanReceiver.java b/htrace-core/src/main/java/org/apache/htrace/impl/POJOSpanReceiver.java
index a77fd39..57e5299 100644
--- a/htrace-core/src/main/java/org/apache/htrace/impl/POJOSpanReceiver.java
+++ b/htrace-core/src/main/java/org/apache/htrace/impl/POJOSpanReceiver.java
@@ -31,8 +31,8 @@ import java.util.HashSet;
 public class POJOSpanReceiver implements SpanReceiver {
   private final Collection<Span> spans;
 
-  @Override
-  public void configure(HTraceConfiguration conf) {
+  public POJOSpanReceiver(HTraceConfiguration conf) {
+    this.spans = new HashSet<Span>();
   }
 
   /**
@@ -42,10 +42,6 @@ public class POJOSpanReceiver implements SpanReceiver {
     return spans;
   }
 
-  public POJOSpanReceiver() {
-    this.spans = new HashSet<Span>();
-  }
-
   @Override
   public void close() throws IOException {
   }
@@ -54,5 +50,4 @@ public class POJOSpanReceiver implements SpanReceiver {
   public void receiveSpan(Span span) {
     spans.add(span);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/htrace-core/src/main/java/org/apache/htrace/impl/StandardOutSpanReceiver.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/main/java/org/apache/htrace/impl/StandardOutSpanReceiver.java b/htrace-core/src/main/java/org/apache/htrace/impl/StandardOutSpanReceiver.java
index 2253f86..014f32b 100644
--- a/htrace-core/src/main/java/org/apache/htrace/impl/StandardOutSpanReceiver.java
+++ b/htrace-core/src/main/java/org/apache/htrace/impl/StandardOutSpanReceiver.java
@@ -26,9 +26,7 @@ import java.io.IOException;
  * Used for testing. Simply prints to standard out any spans it receives.
  */
 public class StandardOutSpanReceiver implements SpanReceiver {
-
-  @Override
-  public void configure(HTraceConfiguration conf) {
+  public StandardOutSpanReceiver(HTraceConfiguration conf) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/htrace-core/src/test/java/org/apache/htrace/TestHTrace.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/test/java/org/apache/htrace/TestHTrace.java b/htrace-core/src/test/java/org/apache/htrace/TestHTrace.java
index ebca7fb..e3d8964 100644
--- a/htrace-core/src/test/java/org/apache/htrace/TestHTrace.java
+++ b/htrace-core/src/test/java/org/apache/htrace/TestHTrace.java
@@ -61,14 +61,14 @@ public class TestHTrace {
       }
       HashMap<String, String> conf = new HashMap<String, String>();
       conf.put("local-file-span-receiver.path", fileName);
-      LocalFileSpanReceiver receiver = new LocalFileSpanReceiver();
-      receiver.configure(HTraceConfiguration.fromMap(conf));
+      LocalFileSpanReceiver receiver =
+          new LocalFileSpanReceiver(HTraceConfiguration.fromMap(conf));
       rcvrs.add(receiver);
     } else {
-      rcvrs.add(new StandardOutSpanReceiver());
+      rcvrs.add(new StandardOutSpanReceiver(HTraceConfiguration.EMPTY));
     }
 
-    POJOSpanReceiver psr = new POJOSpanReceiver();
+    POJOSpanReceiver psr = new POJOSpanReceiver(HTraceConfiguration.EMPTY);
     rcvrs.add(psr);
     runTraceCreatorTraces(new TraceCreator(rcvrs));
 

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/htrace-core/src/test/java/org/apache/htrace/TestSpanReceiverFactory.java
----------------------------------------------------------------------
diff --git a/htrace-core/src/test/java/org/apache/htrace/TestSpanReceiverFactory.java b/htrace-core/src/test/java/org/apache/htrace/TestSpanReceiverFactory.java
new file mode 100644
index 0000000..87b5085
--- /dev/null
+++ b/htrace-core/src/test/java/org/apache/htrace/TestSpanReceiverFactory.java
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.htrace.impl.LocalFileSpanReceiver;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.logging.Level;
+
+public class TestSpanReceiverFactory {
+  /**
+   * Test that if no span receiver is configured, the factory returns null.
+   */
+  @Test
+  public void testGetNullSpanReceiver() {
+    SpanReceiverFactory factory =
+        new SpanReceiverFactory(HTraceConfiguration.EMPTY).logErrors(false);
+    SpanReceiver rcvr = factory.build();
+    Assert.assertEquals(null, rcvr);
+  }
+
+  private static SpanReceiver createSpanReceiver(Map<String, String> m) {
+    HTraceConfiguration hconf = HTraceConfiguration.fromMap(m);
+    SpanReceiverFactory factory =
+        new SpanReceiverFactory(hconf).
+            logErrors(false);
+    return factory.build();
+  }
+
+  /**
+   * Test getting various SpanReceiver objects.
+   */
+  @Test
+  public void testGetSpanReceivers() throws Exception {
+    HashMap<String, String> confMap = new HashMap<String, String>();
+
+    // Create LocalFileSpanReceiver
+    confMap.put(LocalFileSpanReceiver.PATH_KEY, "/tmp/foo");
+    confMap.put(SpanReceiverFactory.SPAN_RECEIVER_CONF_KEY,
+        "org.apache.htrace.impl.LocalFileSpanReceiver");
+    SpanReceiver rcvr = createSpanReceiver(confMap);
+    Assert.assertEquals("org.apache.htrace.impl.LocalFileSpanReceiver",
+        rcvr.getClass().getName());
+    rcvr.close();
+
+    // Create POJOSpanReceiver
+    confMap.remove(LocalFileSpanReceiver.PATH_KEY);
+    confMap.put(SpanReceiverFactory.SPAN_RECEIVER_CONF_KEY, "POJOSpanReceiver");
+    rcvr = createSpanReceiver(confMap);
+    Assert.assertEquals("org.apache.htrace.impl.POJOSpanReceiver",
+        rcvr.getClass().getName());
+    rcvr.close();
+
+    // Create StandardOutSpanReceiver
+    confMap.remove(LocalFileSpanReceiver.PATH_KEY);
+    confMap.put(SpanReceiverFactory.SPAN_RECEIVER_CONF_KEY,
+        "org.apache.htrace.impl.StandardOutSpanReceiver");
+    rcvr = createSpanReceiver(confMap);
+    Assert.assertEquals("org.apache.htrace.impl.StandardOutSpanReceiver",
+        rcvr.getClass().getName());
+    rcvr.close();
+  }
+
+  public static class TestSpanReceiver implements SpanReceiver {
+    final static String SUCCEEDS = "test.span.receiver.succeeds";
+
+    public TestSpanReceiver(HTraceConfiguration conf) {
+      if (conf.get(SUCCEEDS) == null) {
+        throw new RuntimeException("Can't create TestSpanReceiver: " +
+            "invalid configuration.");
+      }
+    }
+
+    @Override
+    public void receiveSpan(Span span) {
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+  }
+
+  /**
+   * Test trying to create a SpanReceiver that experiences an error in the
+   * constructor.
+   */
+  @Test
+  public void testGetSpanReceiverWithConstructorError() throws Exception {
+    HashMap<String, String> confMap = new HashMap<String, String>();
+
+    // Create TestSpanReceiver
+    confMap.put(SpanReceiverFactory.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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/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 4d85e10..4a499ec 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
@@ -123,15 +123,11 @@ public class HBaseSpanReceiver implements SpanReceiver {
   private byte[] icf;
   private int maxSpanBatchSize;
 
-  public HBaseSpanReceiver() {
+  public HBaseSpanReceiver(HTraceConfiguration conf) {
     this.queue = new ArrayBlockingQueue<Span>(1000);
     this.tf = new ThreadFactoryBuilder().setDaemon(true)
                                         .setNameFormat("hbaseSpanReceiver-%d")
                                         .build();
-  }
-
-  @Override
-  public void configure(HTraceConfiguration conf) {
     this.conf = conf;
     this.hconf = HBaseConfiguration.create();
     this.table = Bytes.toBytes(conf.get(TABLE_KEY, DEFAULT_TABLE));

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/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 86f32f7..fdb4da7 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
@@ -130,17 +130,17 @@ public class ZipkinSpanReceiver implements SpanReceiver {
   private String collectorHostname;
   private int collectorPort;
 
-  public ZipkinSpanReceiver() {
+  public ZipkinSpanReceiver(HTraceConfiguration conf) {
     this.queue = new ArrayBlockingQueue<Span>(1000);
     this.protocolFactory = new TBinaryProtocol.Factory();
 
     tf = new ThreadFactoryBuilder().setDaemon(true)
         .setNameFormat("zipkinSpanReceiver-%d")
         .build();
+    configure(conf);
   }
 
-  @Override
-  public void configure(HTraceConfiguration conf) {
+  private void configure(HTraceConfiguration conf) {
     this.conf = conf;
 
     this.collectorHostname = conf.get("zipkin.collector-hostname",

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/f43a0ccb/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 b6b4c43..07a9cea 100644
--- a/htrace-zipkin/src/test/java/org/apache/htrace/TestHTraceSpanToZipkinSpan.java
+++ b/htrace-zipkin/src/test/java/org/apache/htrace/TestHTraceSpanToZipkinSpan.java
@@ -42,7 +42,7 @@ public class TestHTraceSpanToZipkinSpan {
 
   @Test
   public void testHTraceToZipkin() throws IOException {
-    POJOSpanReceiver psr = new POJOSpanReceiver();
+    POJOSpanReceiver psr = new POJOSpanReceiver(HTraceConfiguration.EMPTY);
     Trace.addReceiver(psr);
 
     Span rootSpan = new MilliSpan(ROOT_SPAN_DESC, 1, Span.ROOT_SPAN_ID, 100, "test");