You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@htrace.apache.org by le...@apache.org on 2018/04/11 20:27:00 UTC

[10/12] incubator-htrace git commit: adding rest endpoints for span viewer web app

adding rest endpoints for span viewer web app


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

Branch: refs/heads/master
Commit: 4b2e213ea007d3cadc4475ae5833c57ed3843665
Parents: 0cd394f
Author: Nisala Nirmana <ni...@gmail.com>
Authored: Sat Aug 20 00:08:58 2016 +0530
Committer: Nisala Nirmana <ni...@gmail.com>
Committed: Sat Aug 20 00:08:58 2016 +0530

----------------------------------------------------------------------
 htrace-kudu/pom.xml                             |  12 +
 .../htrace/viewer/KuduClientConstants.java      |  45 +++
 .../apache/htrace/viewer/KuduSpanViewer.java    | 300 +++++++++++++++++++
 .../htrace/viewer/KuduSpanViewerRunner.java     |  38 +++
 .../htrace/viewer/KuduSpanViewerServer.java     |  63 ++++
 .../viewer/KuduSpanViewerSpansServlet.java      |  96 ++++++
 .../viewer/KuduSpanViewerTracesServlet.java     |  88 ++++++
 7 files changed, 642 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/4b2e213e/htrace-kudu/pom.xml
----------------------------------------------------------------------
diff --git a/htrace-kudu/pom.xml b/htrace-kudu/pom.xml
index e977bd9..19195e5 100644
--- a/htrace-kudu/pom.xml
+++ b/htrace-kudu/pom.xml
@@ -33,6 +33,7 @@ language governing permissions and limitations under the License. -->
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <kudu.version>0.9.1</kudu.version>
     <commons.version>1.3.2</commons.version>
+    <jetty.version>9.2.13.v20150730</jetty.version>
     <createDependencyReducedPom>true</createDependencyReducedPom>
   </properties>
 
@@ -156,6 +157,17 @@ language governing permissions and limitations under the License. -->
       <artifactId>commons-io</artifactId>
       <version>${commons.version}</version>
     </dependency>
+
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+      <version>${jetty.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-servlet</artifactId>
+      <version>${jetty.version}</version>
+    </dependency>
   </dependencies>
 
   <profiles>

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/4b2e213e/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduClientConstants.java
----------------------------------------------------------------------
diff --git a/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduClientConstants.java b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduClientConstants.java
new file mode 100644
index 0000000..5945ad6
--- /dev/null
+++ b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduClientConstants.java
@@ -0,0 +1,45 @@
+/*
+ * 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.viewer;
+
+
+public class KuduClientConstants {
+  static final String KUDU_MASTER_HOST_KEY = "kudu.master.host";
+  static final String DEFAULT_KUDU_MASTER_HOST = "127.0.0.1";
+  static final String KUDU_MASTER_PORT_KEY = "kudu.master.port";
+  static final String DEFAULT_KUDU_MASTER_PORT = "7051";
+  static final String DEFAULT_KUDU_SPAN_TABLE = "span";
+  static final String DEFAULT_KUDU_SPAN_TIMELINE_ANNOTATION_TABLE = "span.timeline";
+  static final String DEFAULT_KUDU_COLUMN_SPAN_TRACE_ID = "trace_id";
+  static final String DEFAULT_KUDU_COLUMN_SPAN_START_TIME = "start_time";
+  static final String DEFAULT_KUDU_COLUMN_SPAN_STOP_TIME = "stop_time";
+  static final String DEFAULT_KUDU_COLUMN_SPAN_PARENT_ID_LOW = "parent_id_low";
+  static final String DEFAULT_KUDU_COLUMN_SPAN_PARENT_ID_HIGH = "parent_id_high";
+  static final String DEFAULT_KUDU_COLUMN_SPAN_SPAN_ID = "span_id";
+  static final String DEFAULT_KUDU_COLUMN_SPAN_DESCRIPTION = "description";
+  static final String DEFAULT_KUDU_COLUMN_SPAN_PARENT = "parent";
+  static final String DEFAULT_KUDU_COLUMN_TIMELINE_TIME = "time";;
+  static final String DEFAULT_KUDU_COLUMN_TIMELINE_MESSAGE = "message";
+  static final String DEFAULT_KUDU_COLUMN_TIMELINE_SPANID = "spanid";
+  static final String KUDU_CLIENT_WORKER_COUNT_KEY = "kudu.client.worker.count";
+  static final String KUDU_CLIENT_BOSS_COUNT_KEY = "kudu.client.boss.count";
+  static final String KUDU_CLIENT_STATISTICS_ENABLED_KEY = "kudu.client.statistics.enabled";
+  static final String KUDU_CLIENT_TIMEOUT_ADMIN_OPERATION_KEY = "kudu.client.timeout.admin.operation";
+  static final String KUDU_CLIENT_TIMEOUT_OPERATION_KEY = "kudu.client.timeout.operation";
+  static final String KUDU_CLIENT_TIMEOUT_SOCKET_READ_KEY = "kudu.client.timeout.socket.read";
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/4b2e213e/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewer.java
----------------------------------------------------------------------
diff --git a/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewer.java b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewer.java
new file mode 100644
index 0000000..d3f82d9
--- /dev/null
+++ b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewer.java
@@ -0,0 +1,300 @@
+/*
+ * 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.viewer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.htrace.core.HTraceConfiguration;
+import org.apache.htrace.core.MilliSpan;
+import org.apache.htrace.core.TimelineAnnotation;
+import org.apache.htrace.core.Span;
+import org.apache.htrace.core.SpanId;
+import org.apache.htrace.impl.KuduClientConfiguration;
+import org.kududb.ColumnSchema;
+import org.kududb.Type;
+import org.kududb.client.KuduClient;
+import org.kududb.client.KuduPredicate;
+import org.kududb.client.KuduScanner;
+import org.kududb.client.RowResult;
+import org.kududb.client.RowResultIterator;
+
+import java.io.OutputStreamWriter;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.LinkedList;
+
+public class KuduSpanViewer {
+
+  private static final Log LOG = LogFactory.getLog(KuduSpanViewer.class);
+  private static final String JSON_FIELD_TRACE_ID = "trace_id";
+  private static final String JSON_FIELD_PARENT_ID = "parent_id";
+  private static final String JSON_FIELD_START = "start";
+  private static final String JSON_FIELD_STOP = "stop";
+  private static final String JSON_FIELD_SPAN_ID = "span_id";
+  private static final String JSON_FIELD_DESCRIPTION = "description";
+  private static final String JSON_FIELD_TIMELINE = "timeline";
+  private static final String JSON_FIELD_TIMELINE_TIME = "time";
+  private static final String JSON_FIELD_TIMELINE_MESSEGE = "message";
+  private KuduClient client;
+  private KuduClientConfiguration clientConf;
+
+
+  public KuduSpanViewer(HTraceConfiguration conf) {
+    String masterHost;
+    Integer masterPort;
+    Integer workerCount;
+    Integer bossCount;
+    Boolean isStatisticsEnabled;
+    Long adminOperationTimeout;
+    Long operationTimeout;
+    Long socketReadTimeout;
+    masterHost = conf.get(KuduClientConstants.KUDU_MASTER_HOST_KEY,
+            KuduClientConstants.DEFAULT_KUDU_MASTER_HOST);
+    masterPort = Integer.valueOf(conf.get(KuduClientConstants.KUDU_MASTER_PORT_KEY,
+            KuduClientConstants.DEFAULT_KUDU_MASTER_PORT));
+
+    if (conf.get(KuduClientConstants.KUDU_CLIENT_BOSS_COUNT_KEY) != null) {
+      bossCount = Integer.valueOf(conf.get(KuduClientConstants.KUDU_CLIENT_BOSS_COUNT_KEY));
+    } else {
+      bossCount = null;
+    }
+    if (conf.get(KuduClientConstants.KUDU_CLIENT_WORKER_COUNT_KEY) != null) {
+      workerCount = Integer.valueOf(conf.get(KuduClientConstants.KUDU_CLIENT_WORKER_COUNT_KEY));
+    } else {
+      workerCount = null;
+    }
+    if (conf.get(KuduClientConstants.KUDU_CLIENT_STATISTICS_ENABLED_KEY) != null) {
+      isStatisticsEnabled = Boolean.valueOf(conf.get(KuduClientConstants.KUDU_CLIENT_STATISTICS_ENABLED_KEY));
+    } else {
+      isStatisticsEnabled = null;
+    }
+    if (conf.get(KuduClientConstants.KUDU_CLIENT_TIMEOUT_ADMIN_OPERATION_KEY) != null) {
+      adminOperationTimeout = Long.valueOf(conf.get(KuduClientConstants.KUDU_CLIENT_TIMEOUT_ADMIN_OPERATION_KEY));
+    } else {
+      adminOperationTimeout = null;
+    }
+    if (conf.get(KuduClientConstants.KUDU_CLIENT_TIMEOUT_OPERATION_KEY) != null) {
+      operationTimeout = Long.valueOf(conf.get(KuduClientConstants.KUDU_CLIENT_TIMEOUT_OPERATION_KEY));
+    } else {
+      operationTimeout = null;
+    }
+    if (conf.get(KuduClientConstants.KUDU_CLIENT_TIMEOUT_SOCKET_READ_KEY) != null) {
+      socketReadTimeout = Long.valueOf(conf.get(KuduClientConstants.KUDU_CLIENT_TIMEOUT_SOCKET_READ_KEY));
+    } else {
+      socketReadTimeout = null;
+    }
+    this.clientConf = new KuduClientConfiguration(masterHost,
+            masterPort,
+            workerCount,
+            bossCount,
+            isStatisticsEnabled,
+            adminOperationTimeout,
+            operationTimeout,
+            socketReadTimeout);
+    this.client = clientConf.buildClient();
+  }
+
+  public List<Span> getSpans(long spanId) throws Exception {
+    List<Span> spans = new ArrayList<Span>();
+    List<String> spanColumns = new ArrayList<>();
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_TRACE_ID);
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_SPAN_ID);
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_DESCRIPTION);
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_START_TIME);
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_STOP_TIME);
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_PARENT_ID_HIGH);
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_PARENT_ID_LOW);
+    KuduScanner scanner = client.newScannerBuilder(client.openTable(KuduClientConstants.DEFAULT_KUDU_SPAN_TABLE))
+            .setProjectedColumnNames(spanColumns)
+            .addPredicate(KuduPredicate
+                    .newComparisonPredicate(new ColumnSchema.ColumnSchemaBuilder
+                            (KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_SPAN_ID, Type.INT64)
+                            .build(), KuduPredicate.ComparisonOp.EQUAL, spanId))
+            .build();
+    MilliSpan dbSpan;
+    while (scanner.hasMoreRows()) {
+      RowResultIterator results = scanner.nextRows();
+      while (results.hasNext()) {
+        RowResult result = results.next();
+        long traceId = result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_TRACE_ID);
+        MilliSpan.Builder builder = new MilliSpan.Builder()
+                .spanId(new SpanId(result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_SPAN_ID),
+                        result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_TRACE_ID)))
+                .description(result.getString(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_DESCRIPTION))
+                .begin(result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_START_TIME))
+                .end(result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_STOP_TIME));
+        if (!(result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_PARENT_ID_HIGH) == 0 &&
+                result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_PARENT_ID_LOW) == 0)) {
+          SpanId[] parents = new SpanId[1];
+          parents[0] = new SpanId(result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_PARENT_ID_HIGH),
+                  result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_PARENT_ID_LOW));
+          builder.parents(parents);
+        }
+        List<String> timelineColumns = new ArrayList<>();
+        timelineColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_TIMELINE_TIME);
+        timelineColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_TIMELINE_MESSAGE);
+        KuduScanner timelineScanner = client
+                .newScannerBuilder(client.openTable(KuduClientConstants.DEFAULT_KUDU_SPAN_TIMELINE_ANNOTATION_TABLE))
+                .setProjectedColumnNames(timelineColumns)
+                .addPredicate(KuduPredicate
+                        .newComparisonPredicate(new ColumnSchema.ColumnSchemaBuilder
+                                (KuduClientConstants.DEFAULT_KUDU_COLUMN_TIMELINE_SPANID, Type.INT64)
+                                .build(), KuduPredicate.ComparisonOp.EQUAL, traceId))
+                .build();
+        List<TimelineAnnotation> timelineList = new LinkedList<TimelineAnnotation>();
+        while (timelineScanner.hasMoreRows()) {
+          RowResultIterator timelineResults = timelineScanner.nextRows();
+          while (timelineResults.hasNext()) {
+            RowResult timelineRow = timelineResults.next();
+            timelineList.add(new TimelineAnnotation
+                    (timelineRow.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_TIMELINE_TIME),
+                            timelineRow.getString(KuduClientConstants.DEFAULT_KUDU_COLUMN_TIMELINE_MESSAGE)));
+          }
+        }
+        builder.timeline(timelineList);
+        dbSpan = builder.build();
+        spans.add(dbSpan);
+      }
+    }
+    return spans;
+  }
+
+  public List<Span> getRootSpans() throws Exception {
+    List<Span> spans = new ArrayList<Span>();
+    List<String> spanColumns = new ArrayList<>();
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_TRACE_ID);
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_SPAN_ID);
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_DESCRIPTION);
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_START_TIME);
+    spanColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_STOP_TIME);
+    KuduScanner scanner = client.newScannerBuilder(client.openTable(KuduClientConstants.DEFAULT_KUDU_SPAN_TABLE))
+            .setProjectedColumnNames(spanColumns)
+            .addPredicate(KuduPredicate
+                    .newComparisonPredicate(new ColumnSchema.ColumnSchemaBuilder
+                            (KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_PARENT, Type.BOOL)
+                            .build(), KuduPredicate.ComparisonOp.EQUAL, true))
+            .build();
+    MilliSpan dbSpan;
+    while (scanner.hasMoreRows()) {
+      RowResultIterator results = scanner.nextRows();
+      while (results.hasNext()) {
+        RowResult result = results.next();
+        long traceId = result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_TRACE_ID);
+        MilliSpan.Builder builder = new MilliSpan.Builder()
+                .spanId(new SpanId(result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_SPAN_ID),
+                        result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_TRACE_ID)))
+                .description(result.getString(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_DESCRIPTION))
+                .begin(result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_START_TIME))
+                .end(result.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_SPAN_STOP_TIME));
+        List<String> timelineColumns = new ArrayList<>();
+        timelineColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_TIMELINE_TIME);
+        timelineColumns.add(KuduClientConstants.DEFAULT_KUDU_COLUMN_TIMELINE_MESSAGE);
+        KuduScanner timelineScanner = client
+                .newScannerBuilder(client.openTable(KuduClientConstants.DEFAULT_KUDU_SPAN_TIMELINE_ANNOTATION_TABLE))
+                .setProjectedColumnNames(timelineColumns)
+                .addPredicate(KuduPredicate
+                        .newComparisonPredicate(new ColumnSchema.ColumnSchemaBuilder
+                                (KuduClientConstants.DEFAULT_KUDU_COLUMN_TIMELINE_SPANID, Type.INT64)
+                                .build(), KuduPredicate.ComparisonOp.EQUAL, traceId))
+                .build();
+        List<TimelineAnnotation> timelineList = new LinkedList<TimelineAnnotation>();
+        while (timelineScanner.hasMoreRows()) {
+          RowResultIterator timelineResults = timelineScanner.nextRows();
+          while (timelineResults.hasNext()) {
+            RowResult timelineRow = timelineResults.next();
+            timelineList.add(new TimelineAnnotation
+                    (timelineRow.getLong(KuduClientConstants.DEFAULT_KUDU_COLUMN_TIMELINE_TIME),
+                            timelineRow.getString(KuduClientConstants.DEFAULT_KUDU_COLUMN_TIMELINE_MESSAGE)));
+          }
+        }
+        builder.timeline(timelineList);
+        dbSpan = builder.build();
+        spans.add(dbSpan);
+      }
+    }
+    return spans;
+  }
+
+  public void close() {
+    try {
+      this.client.close();
+    } catch (java.lang.Exception ex) {
+      LOG.error("Couln't close the Kudu DB client connection.", ex);
+    }
+  }
+
+
+  public static String toJsonString(final Span span) throws IOException {
+    ByteArrayOutputStream out = new ByteArrayOutputStream();
+    OutputStreamWriter writer =
+            new OutputStreamWriter(out, Charset.defaultCharset());
+    appendJsonString(span, writer);
+    writer.flush();
+    out.flush();
+    return out.toString();
+  }
+
+  public static void appendJsonString(Span span, OutputStreamWriter writer) throws IOException {
+    writer.append("{");
+    appendField(JSON_FIELD_TRACE_ID, span.getSpanId().getLow(), writer);
+    appendField(JSON_FIELD_SPAN_ID, span.getSpanId().getHigh(), writer);
+    appendField(JSON_FIELD_DESCRIPTION, span.getDescription(), writer);
+    if (span.getParents().length != 0) {
+      appendField(JSON_FIELD_PARENT_ID, span.getParents()[0].getLow(), writer);
+    }
+    appendField(JSON_FIELD_START, span.getStartTimeMillis(), writer);
+    appendField(JSON_FIELD_STOP, span.getStopTimeMillis(), writer);
+    if (!span.getTimelineAnnotations().isEmpty()) {
+      writer.append("\"");
+      writer.append(JSON_FIELD_TIMELINE);
+      writer.append("\"");
+      writer.append(":");
+      writer.append("[");
+      for (TimelineAnnotation annotation : span.getTimelineAnnotations()) {
+        writer.append("{");
+        appendField(JSON_FIELD_TIMELINE_TIME, annotation.getTime(), writer);
+        appendField(JSON_FIELD_TIMELINE_MESSEGE, annotation.getMessage(), writer);
+        writer.append("}");
+      }
+      writer.append("]");
+    }
+    writer.append("}");
+  }
+
+  private static void appendField(String field,
+                                  Object value,
+                                  OutputStreamWriter writer) throws IOException {
+    writer.append("\"");
+    writer.append(field);
+    writer.append("\"");
+    writer.append(":");
+    appendStringValue(value.toString(), writer);
+    writer.append(",");
+  }
+
+  private static void appendStringValue(String value,
+                                        OutputStreamWriter writer) throws IOException {
+    writer.append("\"");
+    writer.append(value.toString());
+    writer.append("\"");
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/4b2e213e/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerRunner.java
----------------------------------------------------------------------
diff --git a/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerRunner.java b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerRunner.java
new file mode 100644
index 0000000..e530c8a
--- /dev/null
+++ b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerRunner.java
@@ -0,0 +1,38 @@
+/*
+ * 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.viewer;
+
+
+import org.apache.htrace.core.HTraceConfiguration;
+
+public class KuduSpanViewerRunner extends Thread {
+  private HTraceConfiguration conf;
+
+  public KuduSpanViewerRunner(HTraceConfiguration conf){
+    this.conf = conf;
+  }
+
+  public void run(){
+    try {
+      new KuduSpanViewerServer().run(conf);
+    }catch (java.lang.Exception ex){
+      //Ignore
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/4b2e213e/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerServer.java
----------------------------------------------------------------------
diff --git a/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerServer.java b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerServer.java
new file mode 100644
index 0000000..d9b6c6a
--- /dev/null
+++ b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerServer.java
@@ -0,0 +1,63 @@
+/*
+ * 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.viewer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.htrace.core.HTraceConfiguration;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.servlet.DefaultServlet;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+
+import java.net.InetSocketAddress;
+import java.net.URI;
+
+public class KuduSpanViewerServer {
+
+  private static final Log LOG = LogFactory.getLog(KuduSpanViewerServer.class);
+  public static final String HTRACE_VIEWER_HTTP_ADDRESS_DEFAULT = "0.0.0.0:17000";
+  private Server server;
+
+  public void stop() throws Exception {
+    if (server != null) {
+      server.stop();
+    }
+    LOG.info("Embedded jetty server stopped successfully.");
+  }
+
+  public int run(HTraceConfiguration conf) throws Exception {
+    URI uri = new URI("http://" + HTRACE_VIEWER_HTTP_ADDRESS_DEFAULT);
+    InetSocketAddress addr = new InetSocketAddress(uri.getHost(), uri.getPort());
+    server = new Server(addr);
+    ServletContextHandler root =
+            new ServletContextHandler(server, "/", ServletContextHandler.SESSIONS);
+    server.setHandler(root);
+    root.addServlet(new ServletHolder(new DefaultServlet()),
+            "/");
+    root.addServlet(new ServletHolder(new KuduSpanViewerTracesServlet(conf)),
+            "/gettraces");
+    root.addServlet(new ServletHolder(new KuduSpanViewerSpansServlet(conf)),
+            "/getspans/*");
+
+    server.start();
+    server.join();
+    return 0;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/4b2e213e/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerSpansServlet.java
----------------------------------------------------------------------
diff --git a/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerSpansServlet.java b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerSpansServlet.java
new file mode 100644
index 0000000..662c21b
--- /dev/null
+++ b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerSpansServlet.java
@@ -0,0 +1,96 @@
+/*
+ * 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.viewer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.htrace.core.HTraceConfiguration;
+import org.apache.htrace.core.Span;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+public class KuduSpanViewerSpansServlet extends HttpServlet {
+
+  private static final Log LOG = LogFactory.getLog(KuduSpanViewerSpansServlet.class);
+  public static final String PREFIX = "/getspans";
+  private static final ThreadLocal<KuduSpanViewer> kuduSpanViewer =
+          new ThreadLocal<KuduSpanViewer>() {
+            @Override
+            protected KuduSpanViewer initialValue() {
+              return null;
+            }
+          };
+  private HTraceConfiguration conf;
+
+  public KuduSpanViewerSpansServlet(HTraceConfiguration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+          throws ServletException, IOException {
+    final String path = request.getRequestURI().substring(PREFIX.length());
+    if (path == null || path.length() == 0) {
+      response.setContentType("text/plain");
+      response.getWriter().print("Invalid input");
+      return;
+    }
+    KuduSpanViewer viewer = kuduSpanViewer.get();
+
+    if (viewer == null) {
+      viewer = new KuduSpanViewer(conf);
+      kuduSpanViewer.set(viewer);
+    }
+    Long traceid = Long.parseLong(path.substring(1));
+    response.setContentType("application/javascript");
+    PrintWriter out = response.getWriter();
+    out.print("[");
+    boolean first = true;
+    try {
+      for (Span span : viewer.getSpans(traceid)) {
+        if (first) {
+          first = false;
+        } else {
+          out.print(",");
+        }
+        out.print(KuduSpanViewer.toJsonString(span));
+      }
+    } catch (java.lang.Exception ex) {
+      LOG.error("Exception occured while retrieving spans from Kudu Backend.");
+    }
+    out.print("]");
+  }
+
+  @Override
+  public void init() throws ServletException {
+  }
+
+  @Override
+  public void destroy() {
+    KuduSpanViewer viewer = kuduSpanViewer.get();
+    if (viewer != null) {
+      viewer.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-htrace/blob/4b2e213e/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerTracesServlet.java
----------------------------------------------------------------------
diff --git a/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerTracesServlet.java b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerTracesServlet.java
new file mode 100644
index 0000000..b296c28
--- /dev/null
+++ b/htrace-kudu/src/main/java/org/apache/htrace/viewer/KuduSpanViewerTracesServlet.java
@@ -0,0 +1,88 @@
+/*
+ * 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.viewer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.htrace.core.HTraceConfiguration;
+import org.apache.htrace.core.Span;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.io.PrintWriter;
+
+
+public class KuduSpanViewerTracesServlet extends HttpServlet {
+  private static final Log LOG = LogFactory.getLog(KuduSpanViewerTracesServlet.class);
+  public static final String PREFIX = "/gettraces";
+  private static final ThreadLocal<KuduSpanViewer> kuduSpanViewer =
+          new ThreadLocal<KuduSpanViewer>() {
+            @Override
+            protected KuduSpanViewer initialValue() {
+              return null;
+            }
+          };
+  private HTraceConfiguration conf;
+
+  public KuduSpanViewerTracesServlet(HTraceConfiguration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void doGet(HttpServletRequest request, HttpServletResponse response)
+          throws ServletException, IOException {
+    KuduSpanViewer viewer = kuduSpanViewer.get();
+    if (viewer == null) {
+      viewer = new KuduSpanViewer(conf);
+      kuduSpanViewer.set(viewer);
+    }
+    response.setContentType("application/javascript");
+    PrintWriter out = response.getWriter();
+    out.print("[");
+    boolean first = true;
+    try {
+      for (Span span : viewer.getRootSpans()) {
+        if (first) {
+          first = false;
+        } else {
+          out.print(",");
+        }
+        out.print(KuduSpanViewer.toJsonString(span));
+      }
+    } catch (java.lang.Exception ex) {
+      LOG.error("Exception occured while retrieving spans from Kudu Backend.");
+    }
+    out.print("]");
+  }
+
+  @Override
+  public void init() throws ServletException {
+  }
+
+  @Override
+  public void destroy() {
+    KuduSpanViewer viewer = kuduSpanViewer.get();
+    if (viewer != null) {
+    viewer.close();
+    }
+  }
+}