You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@avro.apache.org by ph...@apache.org on 2010/08/23 08:57:38 UTC

svn commit: r988004 [1/3] - in /avro/trunk: ./ lang/java/src/java/org/apache/avro/ipc/stats/ lang/java/src/java/org/apache/avro/ipc/trace/ lang/java/src/java/org/apache/avro/ipc/trace/static/ lang/java/src/java/org/apache/avro/ipc/trace/templates/ lang...

Author: philz
Date: Mon Aug 23 06:57:37 2010
New Revision: 988004

URL: http://svn.apache.org/viewvc?rev=988004&view=rev
Log:
AVRO-613. Create basic frontend to view trace results.
Contributed by Patrick Wendell


Added:
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/StaticServlet.java
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceCollection.java
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/avrotrace.js
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/g.bar.js
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/jquery.tipsy.js
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/protovis-r3.2.js
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/tipsy.js
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/templates/
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/templates/collection.vm
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/templates/common.vm
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/templates/node.vm
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/templates/overview.vm
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/templates/traceinput.vm
    avro/trunk/lang/java/src/test/java/org/apache/avro/ipc/trace/TestStaticServlet.java
    avro/trunk/lang/java/src/test/java/org/apache/avro/ipc/trace/TestTraceCollection.java
Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/stats/StaticServlet.java
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/SpanAggregator.java
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/Trace.java
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceClientServlet.java
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceNode.java
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TracePlugin.java
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TracePluginConfiguration.java
    avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/Util.java
    avro/trunk/lang/java/src/test/java/org/apache/avro/ipc/trace/TestBasicTracing.java
    avro/trunk/share/schemas/org/apache/avro/ipc/trace/avroTrace.avpr

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=988004&r1=988003&r2=988004&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Mon Aug 23 06:57:37 2010
@@ -21,6 +21,9 @@ Avro 1.4.0 (unreleased)
 
   NEW FEATURES
 
+    AVRO-613. Create basic frontend to view trace results.
+    (Patrick Wendell via philz)
+
     AVRO-606. Add File-Based Span Storage to TracePlugin
     (Patrick Wendell via philz)
 

Modified: avro/trunk/lang/java/src/java/org/apache/avro/ipc/stats/StaticServlet.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/stats/StaticServlet.java?rev=988004&r1=988003&r2=988004&view=diff
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/stats/StaticServlet.java (original)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/stats/StaticServlet.java Mon Aug 23 06:57:37 2010
@@ -1,3 +1,21 @@
+/**
+ * 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.avro.ipc.stats;
 
 import java.io.IOException;

Modified: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/SpanAggregator.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/SpanAggregator.java?rev=988004&r1=988003&r2=988004&view=diff
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/SpanAggregator.java (original)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/SpanAggregator.java Mon Aug 23 06:57:37 2010
@@ -31,6 +31,11 @@ import edu.emory.mathcs.backport.java.ut
 /**
  * Utility methods for aggregating spans together at various
  * points of trace analysis.
+ * 
+ * The general workflow of trace analysis is:
+ * 
+ * Partial {@link Span}'s --> Complete {@link Span}'s -->
+ * {@link Trace}'s --> {@link TraceCollection}'s
  */
 public class SpanAggregator {
   /**
@@ -110,9 +115,9 @@ public class SpanAggregator {
             for (TimestampedEvent event: other.events) {
               s.events.add(event);
             }
+            s.complete = true;
+            out.completeSpans.add(s);
           }
-          s.complete = true;
-          out.completeSpans.add(s);
         }
       }
     }
@@ -152,4 +157,21 @@ public class SpanAggregator {
     } 
     return out;
   }
+  
+  /**
+   * Given a list of Traces, group traces which share an execution pattern
+   * and return TraceCollection results for each one.
+   */
+  static List<TraceCollection> getTraceCollections(List<Trace> traces) {
+    HashMap<Integer, TraceCollection> collections = 
+      new HashMap<Integer, TraceCollection>();
+    for (Trace t: traces) {
+      if (!collections.containsKey(t.executionPathHash())) {
+        TraceCollection collection = new TraceCollection(t);
+        collections.put(t.executionPathHash(), collection);
+      }
+      collections.get(t.executionPathHash()).addTrace(t);
+    }
+    return new LinkedList<TraceCollection>(collections.values());
+  }
 }

Added: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/StaticServlet.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/StaticServlet.java?rev=988004&view=auto
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/StaticServlet.java (added)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/StaticServlet.java Mon Aug 23 06:57:37 2010
@@ -0,0 +1,50 @@
+/**
+ * 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.avro.ipc.trace;
+
+import java.io.IOException;
+import java.net.URL;
+
+import org.mortbay.jetty.servlet.DefaultServlet;
+import org.mortbay.resource.Resource;
+
+/**
+ * Very simple servlet class capable of serving static files.
+ */
+public class StaticServlet extends DefaultServlet {
+  public Resource getResource(String pathInContext) {
+    // Take only last slice of the URL as a filename, so we can adjust path. 
+    // This also prevents mischief like '../../foo.css'
+    String[] parts = pathInContext.split("/");
+    
+    if (parts.length == 0) {
+      return null;
+    }
+    String filename =  parts[parts.length - 1];
+
+    try {
+      URL resource = getClass().getClassLoader().getResource(
+          "org/apache/avro/ipc/trace/static/" + filename);
+      if (resource == null) { return null; }
+      return Resource.newResource(resource);
+    } catch (IOException e) {
+      return null;
+    }
+  }
+} 

Modified: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/Trace.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/Trace.java?rev=988004&r1=988003&r2=988004&view=diff
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/Trace.java (original)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/Trace.java Mon Aug 23 06:57:37 2010
@@ -228,9 +228,7 @@ public class Trace {  
    */
   private static TraceNode getNode(
       Span s, HashMap<Long, Span> spanRef, HashMap<Long, List<Long>> children) {
-    TraceNode out = new TraceNode();
-    out.span = s;
-    out.children = new LinkedList<TraceNode>();
+    TraceNode out = new TraceNode(s, new LinkedList<TraceNode>());
     
     List<Long> kids = children.get(Util.longValue(s.spanID));
     if (kids == null) { return out; } // no children (base case) 

Modified: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceClientServlet.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceClientServlet.java?rev=988004&r1=988003&r2=988004&view=diff
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceClientServlet.java (original)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceClientServlet.java Mon Aug 23 06:57:37 2010
@@ -19,49 +19,235 @@ package org.apache.avro.ipc.trace;
 
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
 
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.avro.ipc.HttpTransceiver;
+import org.apache.avro.ipc.trace.TraceCollection.TraceNodeStats;
+import org.apache.avro.specific.SpecificRequestor;
+import org.apache.velocity.Template;
+import org.apache.velocity.VelocityContext;
+import org.apache.velocity.app.VelocityEngine;
+import org.apache.velocity.exception.ParseErrorException;
+import org.apache.velocity.exception.ResourceNotFoundException;
+
 public class TraceClientServlet extends HttpServlet {
+  public static Template getTemplate(VelocityEngine ve, String path) 
+    throws IOException {
+    try {
+      return ve.getTemplate(path);
+    } catch (ResourceNotFoundException e) {
+      throw new IOException();
+    } catch (ParseErrorException e) {
+      throw new IOException();
+    } catch (Exception e) {
+      throw new IOException();
+    }
+  }
+  
+  private VelocityEngine velocityEngine;
+  private List<Span> activeSpans;
+  private HashMap<Integer, TraceCollection> activeCollections;
+  private String lastInput = "";
+  
+  public TraceClientServlet() {
+    this.velocityEngine = new VelocityEngine();
+    this.activeCollections = new HashMap<Integer, TraceCollection>();
+    this.activeSpans = new ArrayList<Span>();
+    
+    // These two properties tell Velocity to use its own classpath-based loader
+    velocityEngine.addProperty("resource.loader", "class");
+    velocityEngine.addProperty("class.resource.loader.class",
+        "org.apache.velocity.runtime.resource.loader.ClasspathResourceLoader");
+  }
+  
   public void doPost(HttpServletRequest request, 
     HttpServletResponse response)
     throws ServletException, IOException {
     response.setContentType("text/html");
+    VelocityContext context = new VelocityContext();
+    
     PrintWriter out = response.getWriter();
-
-    out.println("<title>Example</title> <body bgcolor=FFFFFF>");
-    out.println("<h2>Button Clicked</h2>");
-
     String servers = request.getParameter("servers");
 
     if(servers != null){
       String splitToken = System.getProperty("line.separator");
+      lastInput = servers;
       if (splitToken == null) {
         splitToken = "\n";
       }
       String[] parts = servers.split(splitToken);
+      List<URL> urls = new LinkedList<URL>();
       for (String p : parts) {
-        out.println(p + "<br>");
+        String[] portHost = p.split(":");
+        if (portHost.length != 2) { continue; }
+        try {
+          URL url = new URL("http://" + p);
+          urls.add(url);
+        } catch (MalformedURLException e) {
+          continue;
+        }
+      }
+      List<Span> spans = collectAllSpans(urls);
+      List<Span> merged = SpanAggregator.getFullSpans(spans).completeSpans;
+      this.activeSpans.addAll(merged);
+      List<Trace> traces = SpanAggregator.getTraces(merged).traces;
+      List<TraceCollection> collections = 
+        SpanAggregator.getTraceCollections(traces);
+      for (TraceCollection col: collections) {
+        this.activeCollections.put(col.getExecutionPathHash(), col);
       }
+      response.sendRedirect("/overview/");
     } else {
        out.println("No text entered.");
     }
   }
   
+  protected List<Span> collectAllSpans(List<URL> hosts) {
+    List<Span> out = new ArrayList<Span>();
+    for (URL url: hosts) {
+      HttpTransceiver trans = new HttpTransceiver(url);
+      try {
+        AvroTrace client = (AvroTrace) 
+          SpecificRequestor.getClient(AvroTrace.class, trans);
+        for (Span s: client.getAllSpans()) {
+          out.add(s);
+        }
+      }
+      catch (IOException e) {
+        continue;
+      }
+    }
+    return out;
+  }
+  
+  protected List<Span> collectRangedSpans(List<URL> hosts, long start, long end) {
+    List<Span> out = new ArrayList<Span>();
+    for (URL url: hosts) {
+      HttpTransceiver trans = new HttpTransceiver(url);
+      try {
+        AvroTrace client = (AvroTrace) 
+          SpecificRequestor.getClient(AvroTrace.class, trans);
+        for (Span s: client.getSpansInRange(start, end)) {
+          out.add(s);
+        }
+      }
+      catch (IOException e) {
+        continue;
+      }
+    }
+    return out;
+  }
+  
+  /**
+   * We support the following URL patterns
+   * 
+   *   /overview/                          Show all execution patterns
+   *   /collection/[p_id]                  Show details for pattern with [p_id]
+   *   /collection/[p_id]/[n_id]/          Show trace node with [n_id] 
+   */
   public void doGet(HttpServletRequest request, 
       HttpServletResponse response) throws IOException {
     response.setContentType("text/html");
     PrintWriter out = response.getWriter();
+    URL url = new URL(request.getRequestURL().toString());
+    String path = url.getPath();
+    String[] pathParts = path.split("/");
+    
+    if (pathParts.length == 0 || pathParts.length == 1) {
+      loadSpans(out);
+      return;
+    }
+    
+    if (pathParts[1].equals("overview")) {
+      overview(out);
+      return; 
+    }
+    
+    if (pathParts[1].equals("collection")) {
+      if (pathParts.length == 3) {
+        int patternID = Integer.parseInt(pathParts[2]);
+        collection(out, patternID);
+        return;
+      } 
+      else if (pathParts.length == 4) {
+        int patternID = Integer.parseInt(pathParts[2]);
+        int nodeID = Integer.parseInt(pathParts[3]);
+        collectionNode(out, patternID, nodeID);
+        return;
+      }
+      else {
+        response.sendRedirect("/");
+        return;
+      }
+    }
     
-    out.println("<html>");
-    out.println("<head>Form</head>");
-    out.println("<body><form method='post'>");
-    out.println("<textarea name='servers'></textarea>");
-    out.println("<input type='submit' name='submit'>");
-    out.println("</form>");
-    out.println("</html>");
+    // Default
+    response.sendRedirect("/");
+    return;
+  }
+  
+  // VIEW FUNCTIONS
+  
+  /**
+   * Display an overview of patterns detected from a group of spans.
+   */
+  private void overview(PrintWriter out) throws IOException {
+    VelocityContext context = new VelocityContext();
+    context.put("collections", this.activeCollections);
+    context.put("spans", this.activeSpans);
+    Template t = getTemplate(velocityEngine, 
+        "org/apache/avro/ipc/trace/templates/overview.vm");
+    t.merge(context, out);
+  }
+  
+  /**
+   * Display summary statistics for a { @link TraceCollection } of individual
+   * Traces.
+   */
+  private void collection(PrintWriter out, int patternID) throws IOException {
+    VelocityContext context = new VelocityContext();
+    TraceCollection col = this.activeCollections.get(patternID);
+    context.put("collection", col);
+    Template t = getTemplate(velocityEngine, 
+      "org/apache/avro/ipc/trace/templates/collection.vm");
+    t.merge(context, out);
+  }
+  
+  /**
+   * Display in-depth statistics for an individual node of a 
+   * { @link TraceCollection }. 
+   */
+  private void collectionNode(PrintWriter out, int patternID, int nodeID)
+      throws IOException {
+    VelocityContext context = new VelocityContext();
+    TraceCollection col = this.activeCollections.get(patternID);
+    TraceNodeStats node = col.getNodeWithID(nodeID);
+    context.put("collection", col);
+    context.put("node", node);
+    Template t = getTemplate(velocityEngine, 
+      "org/apache/avro/ipc/trace/templates/node.vm");
+    t.merge(context, out);
+  }
+  
+  /**
+   * Display a simple UI for loading span data from remote machines.
+   */
+  private void loadSpans(PrintWriter out) throws IOException {
+    VelocityContext context = new VelocityContext();
+    context.put("last_input", this.lastInput);
+    context.put("default_port", TracePluginConfiguration.DEFAULT_PORT);
+    Template t = getTemplate(velocityEngine, 
+        "org/apache/avro/ipc/trace/templates/traceinput.vm");
+    t.merge(context, out); 
   }
 }

Added: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceCollection.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceCollection.java?rev=988004&view=auto
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceCollection.java (added)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceCollection.java Mon Aug 23 06:57:37 2010
@@ -0,0 +1,281 @@
+/**
+ * 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.avro.ipc.trace;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+
+/**
+ * Provides statistics and representation of a collection of {@link Trace}'s
+ * which share the same execution path.
+ * 
+ * For instance: given the following execution pattern:
+ *   
+ *     b--d
+ *    /
+ *   a
+ *    \
+ *     c--e
+ *     
+ * We might want a report detailing statistics on across several Traces's.
+ * Such as a display of average timing data:
+ * 
+ * [message]         --- = average link time    +++ = average compute time    
+ * a                 <----++++++++++++++++++++++++++++++++++++++++--->
+ *   b                    <----+++++++++++++++++++--->
+ *     d                         <-----++++----->
+ *   c                                                <---+++++->
+ *     e                                                   <+>
+ *     
+ * This requires calculating averages of different timing data at each node
+ * in the call tree.
+ */
+public class TraceCollection {
+  
+  /**
+   * Class to store the timing data associated with a particluar trace.
+   */
+  public class TraceTiming implements Comparable<TraceTiming> {
+    public long preLinkTime;
+    public long computeTime;
+    public long postLinkTime;
+    
+    public TraceTiming(Long preLinkTime, Long computeTime, Long postLinkTime) {
+      this.preLinkTime = preLinkTime;
+      this.computeTime = computeTime;
+      this.postLinkTime = postLinkTime;
+    }
+    
+    public Long getTotalTime() {
+      return new Long(this.preLinkTime + this.computeTime + this.postLinkTime);
+    }
+    
+    public int compareTo(TraceTiming other) {
+      return this.getTotalTime().compareTo(other.getTotalTime());
+    }
+  }
+  
+  /**
+   * Class to store statistics for a particular node in the RPC call tree.
+   */
+  public class TraceNodeStats {
+    CharSequence messageName;
+    List<TraceNodeStats> children;
+    
+    TreeSet<Long> requestPayloads;
+    TreeSet<Long> responsePayloads;
+    
+    // Time can be divided into three components
+    public TreeSet<TraceTiming> traceTimings;
+    
+    /**
+     * Create a TraceNodeStats given a root TraceNode.
+     */
+    public TraceNodeStats(TraceNode root) {
+      this.requestPayloads = new TreeSet<Long>();
+      this.responsePayloads = new TreeSet<Long>();
+      this.traceTimings = new TreeSet<TraceTiming>();
+     
+      this.messageName = root.span.messageName;
+      this.children = new LinkedList<TraceNodeStats>();
+      
+      for (TraceNode tn: root.children) {
+        this.children.add(new TraceNodeStats(tn));
+      }
+    }
+    
+    // Velocity requires getters
+    public SortedSet<Long> getRequestPayloads() { return this.requestPayloads; }
+    public SortedSet<Long> getResponsePayloads() { return this.responsePayloads; }
+    public SortedSet<TraceTiming> getTraceTimings() { return this.traceTimings; }
+    public List<TraceNodeStats> getChildren() { return this.children; }
+    public CharSequence getMessageName() { return this.messageName; } 
+    
+    // Convenience methods for templates
+    public String getAverageTime(SortedSet<TraceTiming> input) {
+      return Util.printableTime(getTimingAverage(input));
+    }
+    public String getMinTime(SortedSet<TraceTiming> input) {
+      return Util.printableTime(input.first().getTotalTime());
+    }
+    public String getMaxTime(SortedSet<TraceTiming> input) {
+      return Util.printableTime(input.last().getTotalTime());
+    }
+ 
+    public String getAverageBytes(SortedSet<Long> input) {
+      return Util.printableBytes(getLongAverage(input));
+    }
+    public String getMinBytes(SortedSet<Long> input) {
+      return Util.printableBytes(input.first());
+    }
+    public String getMaxBytes(SortedSet<Long> input) {
+      return Util.printableBytes(input.last());
+    }
+    
+    public String printBrief() {
+      String out = "'" + messageName + "' ";
+      out += "(Averages) " + "Request Payload: " + 
+        this.getAverageBytes(this.requestPayloads) + " Response Payload: " + 
+        this.getAverageBytes(this.responsePayloads) + " RTT: " +
+        this.getAverageTime(this.traceTimings);
+      return out;
+    }
+    
+  }
+  
+  public TraceNodeStats getNodeWithID(int hashCode) {
+    return getNodeWithIDRecurse(hashCode, this.root);
+  }
+  
+  public TraceNodeStats getNodeWithIDRecurse(int hashCode, TraceNodeStats start) {
+    if (start.hashCode() == hashCode) {
+      return start; // base case, we've found it
+    }
+    else {
+      for (TraceNodeStats tn: start.children) {
+        TraceNodeStats potential = getNodeWithIDRecurse(hashCode, tn);
+        if (potential != null) { return potential; }
+      }
+    }
+    return null;
+  }
+
+  public class TraceComparotor implements Comparator<Trace> {
+    @Override
+    public int compare(Trace o1, Trace o2) {
+      Long rtt1 = o1.getRoot().getPreLinkTime() + 
+                  o1.getRoot().getProcessTime() + 
+                  o1.getRoot().getPostLinkTime(); 
+      Long rtt2 = o2.getRoot().getPreLinkTime() + 
+                  o2.getRoot().getProcessTime() + 
+                  o2.getRoot().getPostLinkTime();
+      return rtt1.compareTo(rtt2);
+    }
+  }
+  
+  private static long getLongAverage(Collection<Long> c) {
+    double out = 0;
+    for (Long l: c) {
+      out += (double) l /c.size(); // Do like this to avoid overflow
+    }
+    return (long) out;
+  }
+  
+  private static long getTimingAverage(Collection<TraceTiming> c) {
+    if (c == null) return 0;
+    double out = 0;
+    for (TraceTiming l: c) {
+      Long val = l.getTotalTime();
+      out += (double) val /c.size(); // Do like this to avoid overflow
+    }
+    return (long) out;
+  }
+  
+  private int exectuionPathHash;
+  private TraceNodeStats root;
+  private TreeSet<Trace> traces; // Store traces, sorted by RTT
+  
+  /**
+   * Create a TraceCollection using the given Trace as a model. Note that 
+   * we do not add this trace to the statistics tracking in this constructor.
+   */
+  public TraceCollection(Trace t) {
+    this.exectuionPathHash = t.executionPathHash();
+    this.root = new TraceNodeStats(t.getRoot());
+    this.traces = new TreeSet<Trace>(new TraceComparotor());
+  }
+  
+  // Getters for Velocity
+  public TraceNodeStats getRootNode() { return this.root; }
+  public int getExecutionPathHash() { return this.exectuionPathHash; }  
+  public TreeSet<Trace> getTraces() { return this.traces; }
+  
+  /**
+   * Returns the [count] longest traces in this collection.
+   */
+  @SuppressWarnings("unchecked")
+  public List<Trace> longestTraces(int count) {
+    TreeSet<Trace> cloned = (TreeSet<Trace>) this.traces.clone();
+    LinkedList<Trace> out = new LinkedList<Trace>();
+    for (int i = 0; i < count; i++) {
+      Trace toAdd = cloned.pollLast();
+      if (toAdd == null) { break; }
+      out.add(toAdd);
+    }
+    return out;
+  }
+  
+  /**
+   * Add a trace to this collection. Timing data from this trace will then be
+   * included in aggregate statistics.
+   */
+  public void addTrace(Trace t) {
+    this.traces.add(t);
+    if (t.executionPathHash() != this.exectuionPathHash) {
+      throw new IllegalArgumentException("Trace added which does not match" +
+          " required execution path.");
+    }
+    recursiveProcess(t.getRoot(), this.root);
+  }
+  
+  private void recursiveProcess(TraceNode tn, TraceNodeStats tns) { 
+    if (tn.children.size() != tns.children.size() ||
+        !tns.messageName.equals(tn.span.messageName)) {
+      throw new IllegalArgumentException("Trace added does not match existing" +
+          "trace");
+    }
+    
+    tns.requestPayloads.add(tn.span.requestPayloadSize);
+    tns.responsePayloads.add(tn.span.responsePayloadSize);
+    tns.traceTimings.add(new TraceTiming(tn.getPreLinkTime(), 
+        tn.getProcessTime(), tn.getPostLinkTime()));
+    tns.messageName = tn.span.messageName;
+
+    for (int i = 0; i < tn.children.size(); i++) {
+      recursiveProcess(tn.children.get(i), tns.children.get(i));
+    }
+  }
+  
+  /**
+   * Print a brief description of this Trace Collection with some summary 
+   * data. Useful for debugging or quick profiling.
+   */
+  public String printBrief() {
+    String out = "TraceCollection:\n";
+    out += printRecurse(this.root, 0);
+    return out;
+  }
+
+  public String printRecurse(TraceNodeStats n, int depth) {
+    String out = "";
+    for (int i = 0; i < depth; i++) {
+      out = out + "  ";
+    }
+    out += n.printBrief() + "\n";
+    for (TraceNodeStats child: n.children) {
+      out = out + printRecurse(child, depth + 1);
+    }
+    return out;
+  }
+}

Modified: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceNode.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceNode.java?rev=988004&r1=988003&r2=988004&view=diff
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceNode.java (original)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TraceNode.java Mon Aug 23 06:57:37 2010
@@ -23,7 +23,7 @@ import java.util.List;
  * A node of of an RPC {@link Trace}. Each node stores a {@link Span} object
  * and a list of zero or more child nodes.
  */
-class TraceNode {
+public class TraceNode {
   /**
    * The {@link Span} to which corresponds to this node in the call tree.
    */
@@ -38,10 +38,6 @@ class TraceNode {
     this.span = span;
     this.children = children;
   }
-  
-  public TraceNode() {
-    
-  }
  
   /** 
    * Return the time stamp associated with a particular SpanEvent in this
@@ -62,7 +58,7 @@ class TraceNode {
    * { @link SpanEvent.SERVER_RECV }. This may be negative or zero in the 
    * case of clock skew.
    */
-  public long getPreNetworkTime() {
+  public long getPreLinkTime() {
     long clientSend = extractEventTime(this, SpanEvent.CLIENT_SEND);
     long serverReceive = extractEventTime(this, SpanEvent.SERVER_RECV);
     
@@ -70,11 +66,18 @@ class TraceNode {
   }
   
   /**
+   * Return pre-link time as a string.
+   */
+  public String getPreLinkTimeString() {
+    return Util.printableTime(this.getPreLinkTime());
+  }
+  
+  /**
    * Return time delta between { @link SpanEvent.SERVER_SEND } and 
    * { @link SpanEvent.CLIENT_RECV }. This may be negative or zero in the 
    * case of clock skew.
    */
-  public long getPostNetworkTime() {
+  public long getPostLinkTime() {
     long serverSend = extractEventTime(this, SpanEvent.SERVER_SEND);
     long clientReceive = extractEventTime(this, SpanEvent.CLIENT_RECV);
     
@@ -82,6 +85,13 @@ class TraceNode {
   }
   
   /**
+   * Return post-link time as a string.
+   */
+  public String getPostLinkTimeString() {
+    return Util.printableTime(this.getPreLinkTime());
+  }
+  
+  /**
    * Return time delta between { @link SpanEvent.SERVER_RECV } and 
    * { @link SpanEvent.SERVER_SEND}.
    */
@@ -91,4 +101,39 @@ class TraceNode {
     
     return serverSend - serverReceive;
   } 
+  
+  /**
+   * Return cpu time as a string.
+   */
+  public String getProcessTimeString() {
+    return Util.printableTime(this.getProcessTime());
+  }
+  
+  /**
+   * Return the children of this node.
+   */
+  public List<TraceNode> getChildren() {
+    return this.children;
+  }
+  
+  // Span data getters for Velicty
+  public String getRequestPayloadSize() { 
+    return Util.printableBytes(this.span.requestPayloadSize);
+  }
+  public String getResponsePayloadSize() { 
+    return Util.printableBytes(this.span.responsePayloadSize);
+  }
+  public String getRequestorHostname() {
+    return this.span.requestorHostname.toString();
+  }
+  public String getResponderHostname() {
+    return this.span.responderHostname.toString();
+  }
+  public String getMessageName() {
+    return this.span.messageName.toString();
+  }
+  public String getLatencyTimeString() {
+    return Util.printableTime(this.getPreLinkTime() + 
+        this.getProcessTime() + this.getPostLinkTime());
+  }
 }

Modified: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TracePlugin.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TracePlugin.java?rev=988004&r1=988003&r2=988004&view=diff
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TracePlugin.java (original)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TracePlugin.java Mon Aug 23 06:57:37 2010
@@ -20,6 +20,8 @@ package org.apache.avro.ipc.trace;
 
 import java.io.IOException;
 import java.net.BindException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Map;
@@ -71,7 +73,6 @@ public class TracePlugin extends RPCPlug
   private static final Utf8 TRACE_ID_KEY = new Utf8("traceID");
   private static final Utf8 SPAN_ID_KEY = new Utf8("spanID");
   private static final Utf8 PARENT_SPAN_ID_KEY = new Utf8("parentSpanID");
-  
 
   class TraceResponder implements AvroTrace {
     private SpanStorage spanStorage;
@@ -124,6 +125,8 @@ public class TracePlugin extends RPCPlug
   // Client interface
   protected Server clientFacingServer;
   
+  private CharSequence hostname;
+  
   public TracePlugin(TracePluginConfiguration conf) throws IOException {
     traceProb = conf.traceProb;
     port = conf.port;
@@ -138,6 +141,12 @@ public class TracePlugin extends RPCPlug
     if (!(clientPort > 0 && clientPort < 65535)) { clientPort = 51200; }
     if (maxSpans < 0) { maxSpans = 5000; }
     
+    try {
+      hostname = InetAddress.getLocalHost().toString();
+    } catch (UnknownHostException e) {
+      hostname = "Unknown";
+    }
+
     currentSpan = new ThreadLocal<Span>(){
       @Override protected Span initialValue(){
           return null;
@@ -172,6 +181,7 @@ public class TracePlugin extends RPCPlug
       // rather than die if port is taken, try to fail over to another port.
       try {
         httpServer = new HttpServer(responder, this.port);
+        httpServer.start();
         bound = true;
       } catch (AvroRuntimeException e) {
         if (e.getCause() instanceof BindException) {
@@ -197,6 +207,7 @@ public class TracePlugin extends RPCPlug
         (RANDOM.nextFloat() < this.traceProb) && enabled) {
       // Start new trace
       Span span = Util.createEventlessSpan(null, null, null);
+      span.requestorHostname = this.hostname;
       this.childSpan.set(span);
     }
     
@@ -204,6 +215,7 @@ public class TracePlugin extends RPCPlug
       Span currSpan = this.currentSpan.get();
       Span span = Util.createEventlessSpan(
           currSpan.traceID, null, currSpan.spanID);   
+      span.requestorHostname = this.hostname;
       this.childSpan.set(span);
     }
     
@@ -244,6 +256,7 @@ public class TracePlugin extends RPCPlug
       traceID.bytes(meta.get(TRACE_ID_KEY).array());
       
       Span span = Util.createEventlessSpan(traceID, spanID, parentSpanID);
+      span.responderHostname = this.hostname;
       
       span.events = new GenericData.Array<TimestampedEvent>(
           100, Schema.createArray(TimestampedEvent.SCHEMA$));
@@ -316,6 +329,8 @@ public class TracePlugin extends RPCPlug
    */
   protected void initializeClientServer() {
     clientFacingServer = new Server();
+    Context staticContext = new Context(clientFacingServer, "/static");
+    staticContext.addServlet(new ServletHolder(new StaticServlet()), "/");
     Context context = new Context(clientFacingServer, "/");
     context.addServlet(new ServletHolder(new TraceClientServlet()), "/");
     boolean connected = false;

Modified: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TracePluginConfiguration.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TracePluginConfiguration.java?rev=988004&r1=988003&r2=988004&view=diff
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TracePluginConfiguration.java (original)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/TracePluginConfiguration.java Mon Aug 23 06:57:37 2010
@@ -24,6 +24,9 @@ import org.apache.avro.ipc.trace.TracePl
  * a common configuration module, wrap this class with your own configuration. 
  */
 public class TracePluginConfiguration {
+  public static final int DEFAULT_CLIENT_PORT = 12345;
+  public static final int DEFAULT_PORT = 12335;
+  
   public double traceProb; // Probability of starting tracing
   public int port;         // Port to serve tracing data
   public int clientPort;   // Port to expose client HTTP interface
@@ -42,8 +45,8 @@ public class TracePluginConfiguration {
    */
   public TracePluginConfiguration() {
     this.traceProb = 0.0;
-    this.port = 12335;
-    this.clientPort = 12345;
+    this.port = DEFAULT_PORT;
+    this.clientPort = DEFAULT_CLIENT_PORT;
     this.storageType = StorageType.MEMORY;
     this.maxSpans = 10000;
     this.enabled = true;

Modified: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/Util.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/Util.java?rev=988004&r1=988003&r2=988004&view=diff
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/Util.java (original)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/Util.java Mon Aug 23 06:57:37 2010
@@ -18,8 +18,6 @@
 
 package org.apache.avro.ipc.trace;
 
-import java.net.InetAddress;
-import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.nio.LongBuffer;
 import java.util.Arrays;
@@ -29,7 +27,6 @@ import java.util.Random;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericData;
-import org.apache.avro.util.Utf8;
 
 /**
  * Utility methods for common tasks in Avro tracing. Mostly consists of
@@ -38,7 +35,6 @@ import org.apache.avro.util.Utf8;
 class Util {
   final private static Random RANDOM = new Random();
   final private static int NANOS_PER_MILI = 1000000;
-  private static Utf8 hostname;
   
   /**
    * Get all SpanEvents contained in Span s.
@@ -102,15 +98,6 @@ class Util {
     
     span.events = new GenericData.Array<TimestampedEvent>(
         10, Schema.createArray(TimestampedEvent.SCHEMA$));
-    
-    if (hostname == null) {
-      try {
-        hostname = new Utf8(InetAddress.getLocalHost().toString());
-      } catch (UnknownHostException e) {
-        hostname = new Utf8("Unknown");
-      }
-    }
-    span.requestorHostname = hostname;
     return span;
   }
   
@@ -166,6 +153,30 @@ class Util {
   }
   
   /**
+   * Convert a timeStamp (in nanoseconds) to a pretty string.
+   */
+  public static String printableTime(long stamp) {
+    String out = "";
+    double milliseconds = (double) stamp / (double) NANOS_PER_MILI;
+    return String.format("%.2fms", milliseconds);
+  }
+  
+  /**
+   * Convert a bytes count to a pretty string.
+   */
+  public static String printableBytes(long bytes) {
+    if (bytes < 1024) {
+      return Long.toString(bytes) + "b";
+    } else if (bytes < (1024 * 1024)) {
+      double kb = (double) bytes / 1024.0;
+      return String.format("%.2fkb", kb);
+    } else {
+      double mb = (double) bytes / (1024.0 * 1024.0);
+      return String.format("%.2fmb", mb);
+    }
+  }
+
+  /**
    * Tests if a span occurred between start and end.
    */
   public static boolean spanInRange(Span s, long start, long end) {

Added: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/avrotrace.js
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/avrotrace.js?rev=988004&view=auto
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/avrotrace.js (added)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/avrotrace.js Mon Aug 23 06:57:37 2010
@@ -0,0 +1,166 @@
+/**
+ * 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.
+ */
+ 
+/* Graph functions for Avro tracing */
+
+function max(one, two) {
+  if (one > two) {
+    return one;
+  }
+  else {
+    return two;
+  }
+}
+
+function makeLatencyChart(data) {
+  var types = ["prelink", "cpu", "postlink"];
+  var totals = data.map(function(d) {return (d.prelink + d.postlink + d.cpu) });
+  
+  /* Sizing and scales. */
+  var w = 400,
+      h = 200,
+      xmax = data.length - 1,
+      ymax = pv.max(totals) / 1000000,
+      x = pv.Scale.linear(0, xmax).range(0, w),
+      y = pv.Scale.linear(0, ymax).range(0, h);
+  /* The root panel. */
+  var vis = new pv.Panel()
+      .width(w)
+      .height(h)
+      .bottom(20)
+      .left(40)
+      .right(10)
+      .top(5);
+  
+  /* X-axis and ticks. */
+  vis.add(pv.Rule)
+      .data(x.ticks())
+      .visible(function(d) d)
+      .left(x)
+      .bottom(-5)
+      .height(5)
+    .anchor("bottom").add(pv.Label)
+      .text(x.tickFormat);
+
+  /* The stack layout. */
+  vis.add(pv.Layout.Stack)
+      .layers(types)
+      .values(data)
+      .x(function(d) x(d.index))
+      .y(function(d, p) y(max(d[p] / 1000000, .001)))
+    .layer.add(pv.Area);
+
+  /* Y-axis and ticks. */
+  vis.add(pv.Rule)
+     .data(y.ticks(5))
+     .bottom(y)
+     .strokeStyle(function(d) d ? "rgba(128,128,128,.2)" : "#000")
+     .anchor("left").add(pv.Label)
+     .text(y.tickFormat);
+     
+  /* Use an invisible panel to capture pan & zoom events. */
+  vis.add(pv.Panel)
+      .events("all")
+      .event("mousewheel", pv.Behavior.zoom())
+      .event("zoom", transform);
+  
+  /** Update the y-scale domains per the new transform. */
+  function transform() {
+    var t = this.transform().invert();
+    y.domain(0, ymax / (1 - t.y/50));
+    vis.add(pv.Rule)
+     .data(y.ticks(5))
+     .bottom(y)
+     .strokeStyle(function(d) d ? "rgba(128,128,128,.2)" : "#000")
+     .anchor("left").add(pv.Label)
+     .text(y.tickFormat);
+    vis.render();
+  }
+     
+  vis.render();
+}
+
+function makePayloadChart(data) {
+  var yVals = data.map(function(d) {return (d.y) });
+  
+  /* Sizing and scales. */
+  var w = 400,
+      h = 200,
+      ymax = pv.max(yVals),
+      x = pv.Scale.linear(data, function(d) d.x).range(0, w),
+      y = pv.Scale.linear(0, ymax).range(0, h);
+  
+  /* The root panel. */
+  var vis = new pv.Panel()
+      .width(w)
+      .height(h)
+      .bottom(20)
+      .left(40)
+      .right(10)
+      .top(5);
+  
+  /* Y-axis and ticks. */
+  vis.add(pv.Rule)
+      .data(y.ticks(5))
+      .bottom(y)
+      .strokeStyle(function(d) d ? "#eee" : "#000")
+    .anchor("left").add(pv.Label)
+      .text(y.tickFormat);
+  
+  /* X-axis and ticks. */
+  vis.add(pv.Rule)
+      .data(x.ticks())
+      .visible(function(d) d)
+      .left(x)
+      .bottom(-5)
+      .height(5)
+    .anchor("bottom").add(pv.Label)
+      .text(x.tickFormat);
+  
+  /* The area with top line. */
+  vis.add(pv.Area)
+      .data(data)
+      .bottom(1)
+      .left(function(d) x(d.x))
+      .height(function(d) y(d.y))
+      .fillStyle("rgb(121,173,210)")
+    .anchor("top").add(pv.Line)
+      .lineWidth(3);
+
+  /* Use an invisible panel to capture pan & zoom events. */
+  vis.add(pv.Panel)
+      .events("all")
+      .event("mousewheel", pv.Behavior.zoom())
+      .event("zoom", transform);
+  
+  /** Update the y-scale domains per the new transform. */
+  function transform() {
+    var t = this.transform().invert();
+    //alert(t.y);
+    y.domain(0, ymax / (1 - t.y/50));
+    vis.add(pv.Rule)
+     .data(y.ticks(5))
+     .bottom(y)
+     .strokeStyle(function(d) d ? "rgba(128,128,128,.2)" : "#000")
+     .anchor("left").add(pv.Label)
+     .text(y.tickFormat);
+    vis.render();
+  }
+  
+  vis.render();
+}
\ No newline at end of file

Added: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/g.bar.js
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/g.bar.js?rev=988004&view=auto
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/g.bar.js (added)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/g.bar.js Mon Aug 23 06:57:37 2010
@@ -0,0 +1,386 @@
+/*!
+ * g.Raphael 0.4.1 - Charting library, based on Raphaël
+ *
+ * Copyright (c) 2009 Dmitry Baranovskiy (http://g.raphaeljs.com)
+ * Licensed under the MIT (http://www.opensource.org/licenses/mit-license.php) license.
+ */
+Raphael.fn.g.barchart = function (x, y, width, height, values, opts) {
+    opts = opts || {};
+    var type = {round: "round", sharp: "sharp", soft: "soft"}[opts.type] || "square",
+        gutter = parseFloat(opts.gutter || "20%"),
+        chart = this.set(),
+        bars = this.set(),
+        covers = this.set(),
+        covers2 = this.set(),
+        total = Math.max.apply(Math, values),
+        stacktotal = [],
+        paper = this,
+        multi = 0,
+        colors = opts.colors || this.g.colors,
+        len = values.length;
+    if (this.raphael.is(values[0], "array")) {
+        total = [];
+        multi = len;
+        len = 0;
+        for (var i = values.length; i--;) {
+            bars.push(this.set());
+            total.push(Math.max.apply(Math, values[i]));
+            len = Math.max(len, values[i].length);
+        }
+        if (opts.stacked) {
+            for (var i = len; i--;) {
+                var tot = 0;
+                for (var j = values.length; j--;) {
+                    tot +=+ values[j][i] || 0;
+                }
+                stacktotal.push(tot);
+            }
+        }
+        for (var i = values.length; i--;) {
+            if (values[i].length < len) {
+                for (var j = len; j--;) {
+                    values[i].push(0);
+                }
+            }
+        }
+        total = Math.max.apply(Math, opts.stacked ? stacktotal : total);
+    }
+    
+    total = (opts.to) || total;
+    var barwidth = width / (len * (100 + gutter) + gutter) * 100,
+        barhgutter = barwidth * gutter / 100,
+        barvgutter = opts.vgutter == null ? 20 : opts.vgutter,
+        stack = [],
+        X = x + barhgutter,
+        Y = (height - 2 * barvgutter) / total;
+    if (!opts.stretch) {
+        barhgutter = Math.round(barhgutter);
+        barwidth = Math.floor(barwidth);
+    }
+    !opts.stacked && (barwidth /= multi || 1);
+    for (var i = 0; i < len; i++) {
+        stack = [];
+        for (var j = 0; j < (multi || 1); j++) {
+            var h = Math.round((multi ? values[j][i] : values[i]) * Y),
+                top = y + height - barvgutter - h,
+                bar = this.g.finger(Math.round(X + barwidth / 2), top + h, barwidth, h, true, type).attr({stroke: "none", fill: colors[multi ? j : i]});
+            if (multi) {
+                bars[j].push(bar);
+            } else {
+                bars.push(bar);
+            }
+            bar.y = top;
+            bar.x = Math.round(X + barwidth / 2);
+            bar.w = barwidth;
+            bar.h = h;
+            bar.value = multi ? values[j][i] : values[i];
+            if (!opts.stacked) {
+                X += barwidth;
+            } else {
+                stack.push(bar);
+            }
+        }
+        if (opts.stacked) {
+            var cvr;
+            covers2.push(cvr = this.rect(stack[0].x - stack[0].w / 2, y, barwidth, height).attr(this.g.shim));
+            cvr.bars = this.set();
+            var size = 0;
+            for (var s = stack.length; s--;) {
+                stack[s].toFront();
+            }
+            for (var s = 0, ss = stack.length; s < ss; s++) {
+                var bar = stack[s],
+                    cover,
+                    h = (size + bar.value) * Y,
+                    path = this.g.finger(bar.x, y + height - barvgutter - !!size * .5, barwidth, h, true, type, 1);
+                cvr.bars.push(bar);
+                size && bar.attr({path: path});
+                bar.h = h;
+                bar.y = y + height - barvgutter - !!size * .5 - h;
+                covers.push(cover = this.rect(bar.x - bar.w / 2, bar.y, barwidth, bar.value * Y).attr(this.g.shim));
+                cover.bar = bar;
+                cover.value = bar.value;
+                size += bar.value;
+            }
+            X += barwidth;
+        }
+        X += barhgutter;
+    }
+    covers2.toFront();
+    X = x + barhgutter;
+    if (!opts.stacked) {
+        for (var i = 0; i < len; i++) {
+            for (var j = 0; j < (multi || 1); j++) {
+                var cover;
+                covers.push(cover = this.rect(Math.round(X), y + barvgutter, barwidth, height - barvgutter).attr(this.g.shim));
+                cover.bar = multi ? bars[j][i] : bars[i];
+                cover.value = cover.bar.value;
+                X += barwidth;
+            }
+            X += barhgutter;
+        }
+    }
+    chart.label = function (labels, isBottom) {
+        labels = labels || [];
+        this.labels = paper.set();
+        var L, l = -Infinity;
+        if (opts.stacked) {
+            for (var i = 0; i < len; i++) {
+                var tot = 0;
+                for (var j = 0; j < (multi || 1); j++) {
+                    tot += multi ? values[j][i] : values[i];
+                    if (j == multi - 1) {
+                        var label = paper.g.labelise(labels[i], tot, total);
+                        L = paper.g.text(bars[i * (multi || 1) + j].x, y + height - barvgutter / 2, label).insertBefore(covers[i * (multi || 1) + j]);
+                        var bb = L.getBBox();
+                        if (bb.x - 7 < l) {
+                            L.remove();
+                        } else {
+                            this.labels.push(L);
+                            l = bb.x + bb.width;
+                        }
+                    }
+                }
+            }
+        } else {
+            for (var i = 0; i < len; i++) {
+                for (var j = 0; j < (multi || 1); j++) {
+                    var label = paper.g.labelise(multi ? labels[j] && labels[j][i] : labels[i], multi ? values[j][i] : values[i], total);
+                    L = paper.g.text(bars[i * (multi || 1) + j].x, isBottom ? y + height - barvgutter / 2 : bars[i * (multi || 1) + j].y - 10, label).insertBefore(covers[i * (multi || 1) + j]);
+                    var bb = L.getBBox();
+                    if (bb.x - 7 < l) {
+                        L.remove();
+                    } else {
+                        this.labels.push(L);
+                        l = bb.x + bb.width;
+                    }
+                }
+            }
+        }
+        return this;
+    };
+    chart.hover = function (fin, fout) {
+        covers2.hide();
+        covers.show();
+        covers.mouseover(fin).mouseout(fout);
+        return this;
+    };
+    chart.hoverColumn = function (fin, fout) {
+        covers.hide();
+        covers2.show();
+        fout = fout || function () {};
+        covers2.mouseover(fin).mouseout(fout);
+        return this;
+    };
+    chart.click = function (f) {
+        covers2.hide();
+        covers.show();
+        covers.click(f);
+        return this;
+    };
+    chart.each = function (f) {
+        if (!Raphael.is(f, "function")) {
+            return this;
+        }
+        for (var i = covers.length; i--;) {
+            f.call(covers[i]);
+        }
+        return this;
+    };
+    chart.eachColumn = function (f) {
+        if (!Raphael.is(f, "function")) {
+            return this;
+        }
+        for (var i = covers2.length; i--;) {
+            f.call(covers2[i]);
+        }
+        return this;
+    };
+    chart.clickColumn = function (f) {
+        covers.hide();
+        covers2.show();
+        covers2.click(f);
+        return this;
+    };
+    chart.push(bars, covers, covers2);
+    chart.bars = bars;
+    chart.covers = covers;
+    return chart;
+};
+Raphael.fn.g.hbarchart = function (x, y, width, height, values, opts) {
+    opts = opts || {};
+    var type = {round: "round", sharp: "sharp", soft: "soft"}[opts.type] || "square",
+        gutter = parseFloat(opts.gutter || "20%"),
+        chart = this.set(),
+        bars = this.set(),
+        covers = this.set(),
+        covers2 = this.set(),
+        total = Math.max.apply(Math, values),
+        stacktotal = [],
+        paper = this,
+        multi = 0,
+        colors = opts.colors || this.g.colors,
+        len = values.length;
+    if (this.raphael.is(values[0], "array")) {
+        total = [];
+        multi = len;
+        len = 0;
+        for (var i = values.length; i--;) {
+            bars.push(this.set());
+            total.push(Math.max.apply(Math, values[i]));
+            len = Math.max(len, values[i].length);
+        }
+        if (opts.stacked) {
+            for (var i = len; i--;) {
+                var tot = 0;
+                for (var j = values.length; j--;) {
+                    tot +=+ values[j][i] || 0;
+                }
+                stacktotal.push(tot);
+            }
+        }
+        for (var i = values.length; i--;) {
+            if (values[i].length < len) {
+                for (var j = len; j--;) {
+                    values[i].push(0);
+                }
+            }
+        }
+        total = Math.max.apply(Math, opts.stacked ? stacktotal : total);
+    }
+    
+    total = (opts.to) || total;
+    var barheight = Math.floor(height / (len * (100 + gutter) + gutter) * 100),
+        bargutter = Math.floor(barheight * gutter / 100),
+        stack = [],
+        Y = y + bargutter,
+        X = (width - 1) / total;
+    !opts.stacked && (barheight /= multi || 1);
+    for (var i = 0; i < len; i++) {
+        stack = [];
+        for (var j = 0; j < (multi || 1); j++) {
+            var val = multi ? values[j][i] : values[i],
+                bar = this.g.finger(x, Y + barheight / 2, Math.round(val * X), barheight - 1, false, type).attr({stroke: "none", fill: colors[multi ? j : i]});
+            if (multi) {
+                bars[j].push(bar);
+            } else {
+                bars.push(bar);
+            }
+            bar.x = x + Math.round(val * X);
+            bar.y = Y + barheight / 2;
+            bar.w = Math.round(val * X);
+            bar.h = barheight;
+            bar.value = +val;
+            if (!opts.stacked) {
+                Y += barheight;
+            } else {
+                stack.push(bar);
+            }
+        }
+        if (opts.stacked) {
+            var cvr = this.rect(x, stack[0].y - stack[0].h / 2, width, barheight).attr(this.g.shim);
+            covers2.push(cvr);
+            cvr.bars = this.set();
+            var size = 0;
+            for (var s = stack.length; s--;) {
+                stack[s].toFront();
+            }
+            for (var s = 0, ss = stack.length; s < ss; s++) {
+                var bar = stack[s],
+                    cover,
+                    val = Math.round((size + bar.value) * X),
+                    path = this.g.finger(x, bar.y, val, barheight - 1, false, type, 1);
+                cvr.bars.push(bar);
+                size && bar.attr({path: path});
+                bar.w = val;
+                bar.x = x + val;
+                covers.push(cover = this.rect(x + size * X, bar.y - bar.h / 2, bar.value * X, barheight).attr(this.g.shim));
+                cover.bar = bar;
+                size += bar.value;
+            }
+            Y += barheight;
+        }
+        Y += bargutter;
+    }
+    covers2.toFront();
+    Y = y + bargutter;
+    if (!opts.stacked) {
+        for (var i = 0; i < len; i++) {
+            for (var j = 0; j < (multi || 1); j++) {
+                var cover = this.rect(x, Y, width, barheight).attr(this.g.shim);
+                covers.push(cover);
+                cover.bar = multi ? bars[j][i] : bars[i];
+                cover.value = cover.bar.value;
+                Y += barheight;
+            }
+            Y += bargutter;
+        }
+    }
+    chart.label = function (labels, isRight) {
+        labels = labels || [];
+        this.labels = paper.set();
+        for (var i = 0; i < len; i++) {
+            for (var j = 0; j < multi; j++) {
+                var  label = paper.g.labelise(multi ? labels[j] && labels[j][i] : labels[i], multi ? values[j][i] : values[i], total);
+                var X = isRight ? bars[i * (multi || 1) + j].x - barheight / 2 + 3 : x + 5,
+                    A = isRight ? "end" : "start",
+                    L;
+                this.labels.push(L = paper.g.text(X, bars[i * (multi || 1) + j].y, label).attr({"text-anchor": A}).insertBefore(covers[0]));
+                if (L.getBBox().x < x + 5) {
+                    L.attr({x: x + 5, "text-anchor": "start"});
+                } else {
+                    bars[i * (multi || 1) + j].label = L;
+                }
+            }
+        }
+        return this;
+    };
+    chart.hover = function (fin, fout) {
+        covers2.hide();
+        covers.show();
+        fout = fout || function () {};
+        covers.mouseover(fin).mouseout(fout);
+        return this;
+    };
+    chart.hoverColumn = function (fin, fout) {
+        covers.hide();
+        covers2.show();
+        fout = fout || function () {};
+        covers2.mouseover(fin).mouseout(fout);
+        return this;
+    };
+    chart.each = function (f) {
+        if (!Raphael.is(f, "function")) {
+            return this;
+        }
+        for (var i = covers.length; i--;) {
+            f.call(covers[i]);
+        }
+        return this;
+    };
+    chart.eachColumn = function (f) {
+        if (!Raphael.is(f, "function")) {
+            return this;
+        }
+        for (var i = covers2.length; i--;) {
+            f.call(covers2[i]);
+        }
+        return this;
+    };
+    chart.click = function (f) {
+        covers2.hide();
+        covers.show();
+        covers.click(f);
+        return this;
+    };
+    chart.clickColumn = function (f) {
+        covers.hide();
+        covers2.show();
+        covers2.click(f);
+        return this;
+    };
+    chart.push(bars, covers, covers2);
+    chart.bars = bars;
+    chart.covers = covers;
+    return chart;
+};

Added: avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/jquery.tipsy.js
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/jquery.tipsy.js?rev=988004&view=auto
==============================================================================
--- avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/jquery.tipsy.js (added)
+++ avro/trunk/lang/java/src/java/org/apache/avro/ipc/trace/static/jquery.tipsy.js Mon Aug 23 06:57:37 2010
@@ -0,0 +1,125 @@
+/*!
+ * The MIT License
+ * 
+ * Copyright (c) 2008 Jason Frame (jason@onehackoranother.com)
+ * 
+ * Permission is hereby granted, free of charge, to any person obtaining a copy
+ * of this software and associated documentation files (the "Software"), to deal
+ * in the Software without restriction, including without limitation the rights
+ * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+ * copies of the Software, and to permit persons to whom the Software is
+ * furnished to do so, subject to the following conditions:
+ * 
+ * The above copyright notice and this permission notice shall be included in
+ * all copies or substantial portions of the Software.
+ * 
+ * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+ * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+ * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+ * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+ * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+ * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+ * THE SOFTWARE.
+ */
+(function($) {
+    $.fn.tipsy = function(options) {
+        options = $.extend({}, $.fn.tipsy.defaults, options);
+        
+        return this.each(function() {
+            
+            var opts = $.fn.tipsy.elementOptions(this, options);
+            
+            $(this).hover(function() {
+
+                $.data(this, 'cancel.tipsy', true);
+
+                var tip = $.data(this, 'active.tipsy');
+                if (!tip) {
+                    tip = $('<div class="tipsy"><div class="tipsy-inner"/></div>');
+                    tip.css({position: 'absolute', zIndex: 100000});
+                    $.data(this, 'active.tipsy', tip);
+                }
+
+                if ($(this).attr('title') || typeof($(this).attr('original-title')) != 'string') {
+                    $(this).attr('original-title', $(this).attr('title') || '').removeAttr('title');
+                }
+                var title;
+                if (typeof opts.title == 'string') {
+                    title = $(this).attr(opts.title == 'title' ? 'original-title' : opts.title);
+                } else if (typeof opts.title == 'function') {
+                    title = opts.title.call(this);
+                }
+
+                tip.find('.tipsy-inner')[opts.html ? 'html' : 'text'](title || opts.fallback);
+
+                var pos = $.extend({}, $(this).offset(), {width: this.offsetWidth, height: this.offsetHeight});
+                tip.get(0).className = 'tipsy'; // reset classname in case of dynamic gravity
+                tip.remove().css({top: 0, left: 0, visibility: 'hidden', display: 'block'}).appendTo(document.body);
+                var actualWidth = tip[0].offsetWidth, actualHeight = tip[0].offsetHeight;
+                var gravity = (typeof opts.gravity == 'function') ? opts.gravity.call(this) : opts.gravity;
+
+                switch (gravity.charAt(0)) {
+                    case 'n':
+                        tip.css({top: pos.top + pos.height, left: pos.left + pos.width / 2 - actualWidth / 2}).addClass('tipsy-north');
+                        break;
+                    case 's':
+                        tip.css({top: pos.top - actualHeight, left: pos.left + pos.width / 2 - actualWidth / 2}).addClass('tipsy-south');
+                        break;
+                    case 'e':
+                        tip.css({top: pos.top + pos.height / 2 - actualHeight / 2, left: pos.left - actualWidth}).addClass('tipsy-east');
+                        break;
+                    case 'w':
+                        tip.css({top: pos.top + pos.height / 2 - actualHeight / 2, left: pos.left + pos.width}).addClass('tipsy-west');
+                        break;
+                }
+
+                if (opts.fade) {
+                    tip.css({opacity: 0, display: 'block', visibility: 'visible'}).animate({opacity: 0.8});
+                } else {
+                    tip.css({visibility: 'visible'});
+                }
+
+            }, function() {
+                $.data(this, 'cancel.tipsy', false);
+                var self = this;
+                setTimeout(function() {
+                    if ($.data(this, 'cancel.tipsy')) return;
+                    var tip = $.data(self, 'active.tipsy');
+                    if (opts.fade) {
+                        tip.stop().fadeOut(function() { $(this).remove(); });
+                    } else {
+                        tip.remove();
+                    }
+                }, 100);
+
+            });
+            
+        });
+        
+    };
+    
+    // Overwrite this method to provide options on a per-element basis.
+    // For example, you could store the gravity in a 'tipsy-gravity' attribute:
+    // return $.extend({}, options, {gravity: $(ele).attr('tipsy-gravity') || 'n' });
+    // (remember - do not modify 'options' in place!)
+    $.fn.tipsy.elementOptions = function(ele, options) {
+        return $.metadata ? $.extend({}, options, $(ele).metadata()) : options;
+    };
+    
+    $.fn.tipsy.defaults = {
+        fade: false,
+        fallback: '',
+        gravity: 'n',
+        html: false,
+        title: 'title'
+    };
+    
+    $.fn.tipsy.autoNS = function() {
+        return $(this).offset().top > ($(document).scrollTop() + $(window).height() / 2) ? 's' : 'n';
+    };
+    
+    $.fn.tipsy.autoWE = function() {
+        return $(this).offset().left > ($(document).scrollLeft() + $(window).width() / 2) ? 'e' : 'w';
+    };
+    
+})(jQuery);