You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@chukwa.apache.org by ey...@apache.org on 2009/12/30 05:30:24 UTC

svn commit: r894540 - in /hadoop/chukwa/trunk: ./ conf/ src/java/org/apache/hadoop/chukwa/datacollection/writer/ src/java/org/apache/hadoop/chukwa/dataloader/ src/java/org/apache/hadoop/chukwa/rest/bean/ src/java/org/apache/hadoop/chukwa/rest/resource/...

Author: eyang
Date: Wed Dec 30 04:30:23 2009
New Revision: 894540

URL: http://svn.apache.org/viewvc?rev=894540&view=rev
Log:
CHUKWA-441. Added real time Hadoop activity monitor. (Eric Yang)

Added:
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/dataloader/SocketDataLoader.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/rest/bean/ClientTraceBean.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/rest/resource/ClientTrace.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/dataloader/TestSocketDataLoader.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/rest/resource/TestClientTrace.java
    hadoop/chukwa/trunk/src/web/hicc/descriptors/hadoop_activity.descriptor
    hadoop/chukwa/trunk/src/web/hicc/js/activity.js
    hadoop/chukwa/trunk/src/web/hicc/js/processing.js
    hadoop/chukwa/trunk/src/web/hicc/jsp/hadoop_activity.jsp
Modified:
    hadoop/chukwa/trunk/CHANGES.txt
    hadoop/chukwa/trunk/build.xml
    hadoop/chukwa/trunk/conf/chukwa-collector-conf.xml.template
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/writer/SocketTeeWriter.java

Modified: hadoop/chukwa/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/CHANGES.txt?rev=894540&r1=894539&r2=894540&view=diff
==============================================================================
--- hadoop/chukwa/trunk/CHANGES.txt (original)
+++ hadoop/chukwa/trunk/CHANGES.txt Wed Dec 30 04:30:23 2009
@@ -4,6 +4,8 @@
 
   NEW FEATURES
 
+    CHUKWA-441. Added real time Hadoop activity monitor. (Eric Yang)
+
     CHUKWA-433. File-per-post writer for benchmark purposes. (asrabkin)
 
     CHUKWA-431. UDP Adaptor. (asrabkin)

Modified: hadoop/chukwa/trunk/build.xml
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/build.xml?rev=894540&r1=894539&r2=894540&view=diff
==============================================================================
--- hadoop/chukwa/trunk/build.xml (original)
+++ hadoop/chukwa/trunk/build.xml Wed Dec 30 04:30:23 2009
@@ -485,6 +485,7 @@
                     <include name="hicc.war" />
                   </fileset>
                 </copy>
+                <echo message="localhost" file="${test.build.dir}/conf/collectors"/>
                 <copy file="${conf.dir}/chukwa-common.xml.template" tofile="${test.build.dir}/conf/chukwa-common.xml">
 			<filterchain>
 				<replacetokens>

Modified: hadoop/chukwa/trunk/conf/chukwa-collector-conf.xml.template
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/conf/chukwa-collector-conf.xml.template?rev=894540&r1=894539&r2=894540&view=diff
==============================================================================
--- hadoop/chukwa/trunk/conf/chukwa-collector-conf.xml.template (original)
+++ hadoop/chukwa/trunk/conf/chukwa-collector-conf.xml.template Wed Dec 30 04:30:23 2009
@@ -21,7 +21,15 @@
 
 <configuration>
 
+  <property>
+    <name>chukwaCollector.writerClass</name>
+    <value>org.apache.hadoop.chukwa.datacollection.writer.PipelineStageWriter</value>
+  </property>
 
+  <property>
+    <name>chukwaCollector.pipeline</name>
+    <value>org.apache.hadoop.chukwa.datacollection.writer.SocketTeeWriter,org.apache.hadoop.chukwa.datacollection.writer.SeqFileWriter</value>
+  </property>
 
 <!-- LocalWriter parameters
   <property>

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java?rev=894540&r1=894539&r2=894540&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java Wed Dec 30 04:30:23 2009
@@ -44,7 +44,7 @@
  * this object.
  * 
  */
-public class SeqFileWriter implements ChukwaWriter {
+public class SeqFileWriter extends PipelineableWriter implements ChukwaWriter {
   static Logger log = Logger.getLogger(SeqFileWriter.class);
   public static boolean ENABLE_ROTATION_ON_CLOSE = true;
 

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/writer/SocketTeeWriter.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/writer/SocketTeeWriter.java?rev=894540&r1=894539&r2=894540&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/writer/SocketTeeWriter.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/writer/SocketTeeWriter.java Wed Dec 30 04:30:23 2009
@@ -66,7 +66,7 @@
   static enum DataFormat {Raw, Writable, Header};
   
   static boolean USE_KEEPALIVE = true;
-  static final int DEFAULT_PORT = 9094;
+  public static final int DEFAULT_PORT = 9094;
   static int QUEUE_LENGTH = 1000;
   
   static Logger log = Logger.getLogger(SocketTeeWriter.class);

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/dataloader/SocketDataLoader.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/dataloader/SocketDataLoader.java?rev=894540&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/dataloader/SocketDataLoader.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/dataloader/SocketDataLoader.java Wed Dec 30 04:30:23 2009
@@ -0,0 +1,190 @@
+/*
+ * 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.hadoop.chukwa.dataloader;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.net.Socket;
+import java.net.SocketException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.chukwa.Chunk;
+import org.apache.hadoop.chukwa.ChunkImpl;
+import org.apache.hadoop.chukwa.conf.ChukwaConfiguration;
+import org.apache.hadoop.chukwa.datacollection.DataFactory;
+import org.apache.hadoop.chukwa.datacollection.writer.SocketTeeWriter;
+import org.apache.hadoop.chukwa.util.ExceptionUtil;
+import org.apache.log4j.Logger;
+
+/**
+ * Socket Data Loader, also known as the SDL, is a framework for allowing direct
+ * access to log data under the Chukwa Collector in a safe and efficient manner.
+ * Subscribe to chukwaCollector.tee.port for data streaming.
+ * Defaults socket tee port is 9094.
+ */
+public class SocketDataLoader implements Runnable {
+  private String hostname = "localhost";
+  private int port = 9094;
+  private static Logger log = Logger.getLogger(SocketDataLoader.class);
+  private Socket s = null;
+  private DataInputStream dis = null;
+  private DataOutputStream dos = null;
+  private Queue<Chunk> q = new LinkedList<Chunk>();
+  private String recordType = null;
+  private boolean running = false;
+  private static final int QUEUE_MAX = 10;
+  private Iterator<String> collectors = null;
+  private static Pattern pattern = Pattern.compile("(.+?)\\://(.+?)\\:(.+?)");
+
+  /*
+   * Create and start an instance of SocketDataLoader.
+   * @param Record Type
+   */
+  public SocketDataLoader(String recordType) {
+    this.recordType = recordType;
+    try {
+      collectors = DataFactory.getInstance().getCollectorURLs(new ChukwaConfiguration());
+    } catch (IOException e) {
+      log.error(ExceptionUtil.getStackTrace(e));
+    }
+    Matcher m = pattern.matcher(collectors.next());
+    // Socket data loader only supports to stream data from a single collector.  
+    // For large deployment, it may require to setup multi-tiers of collectors to
+    // channel data into a single collector for display.
+    if(m.matches()) {
+      hostname = m.group(2);
+    }
+    start();
+  }
+  
+  /*
+   * Establish a connection to chukwa collector and filter data stream
+   * base on record type.
+   */
+  public synchronized void start() {
+    try {
+      running = true;
+      s = new Socket(hostname, port);
+      try {
+        s.setSoTimeout(120000);
+        dos = new DataOutputStream (s.getOutputStream());
+        StringBuilder output = new StringBuilder();
+        output.append(SocketTeeWriter.WRITABLE);
+        if(recordType.toLowerCase().intern()!="all".intern()) {
+          output.append(" datatype=");
+          output.append(recordType);
+        } else {
+          output.append(" all");
+        }
+        output.append("\n");
+        dos.write((output.toString()).getBytes());
+      } catch (SocketException e) {
+        log.warn("Error while settin soTimeout to 120000");
+      }
+      dis = new DataInputStream(s
+          .getInputStream());
+      dis.readFully(new byte[3]); //read "OK\n"
+      StringBuilder sb = new StringBuilder();
+      sb.append("Subscribe to ");
+      sb.append(hostname);
+      sb.append(":");
+      sb.append(port);
+      sb.append(" for record type: ");
+      sb.append(recordType);
+      log.info(sb.toString());
+      Thread t=new Thread (this);
+      t.start();
+    } catch (IOException e) {
+      log.error(ExceptionUtil.getStackTrace(e));
+      stop();
+    }    
+  }
+  
+  /*
+   * Read the current chunks in the SDL queue.
+   * @return List of chunks in the SDL queue.
+   */
+  public synchronized Collection<Chunk> read() throws NoSuchElementException {
+    Collection<Chunk> list = Collections.synchronizedCollection(q);
+    return list;
+  }
+  
+  /*
+   * Unsubscribe from Chukwa collector and stop streaming.
+   */
+  public void stop() {
+    if(s!=null) {
+      try {
+        dis.close();
+        dos.close();
+        s.close();
+        StringBuilder sb = new StringBuilder();
+        sb.append("Unsubscribe from ");
+        sb.append(hostname);
+        sb.append(":");
+        sb.append(port);
+        sb.append(" for data type: ");
+        sb.append(recordType);
+        log.info(sb.toString());
+        running = false;
+      } catch (IOException e) {
+        log.debug("Unable to close Socket Tee client socket.");
+      }
+    }
+  }
+
+  /*
+   * Check if streaming is currently happening for the current instance of SDL.
+   * @return running state of the SDL,
+   */
+  public boolean running() {
+    return running;
+  }
+  
+  /*
+   * Background thread for reading data from SocketTeeWriter, and add new data
+   * into SDL queue.
+   */
+  @Override
+  public void run() {
+    try {
+      Chunk c;
+      while ((c = ChunkImpl.read(dis)) != null) {
+        StringBuilder sb = new StringBuilder();
+        sb.append("Chunk received, recordType:");
+        sb.append(c.getDataType());
+        log.debug(sb);
+        if(q.size()>QUEUE_MAX) {
+          q.poll();
+        }
+        q.offer(c);
+      }
+    } catch (IOException e) {
+      log.error(ExceptionUtil.getStackTrace(e));
+      stop();
+    }
+  }
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/rest/bean/ClientTraceBean.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/rest/bean/ClientTraceBean.java?rev=894540&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/rest/bean/ClientTraceBean.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/rest/bean/ClientTraceBean.java Wed Dec 30 04:30:23 2009
@@ -0,0 +1,77 @@
+/*
+ * 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.hadoop.chukwa.rest.bean;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement
+public class ClientTraceBean {
+  
+  private String action;
+  private long size = 0L;
+  private String src;
+  private String dest;
+  private String date;
+  
+  @XmlElement
+  public String getAction() {
+    return action;
+  }
+  
+  public void setAction(String action) {
+    this.action = action;
+  }
+  
+  @XmlElement
+  public String getSrc() {
+    return src;
+  }
+  
+  public void setSrc(String src) {
+    this.src = src;
+  }
+  
+  @XmlElement
+  public String getDest() {
+    return dest;
+  }
+  
+  public void setDest(String dest) {
+    this.dest = dest;
+  }
+  
+  @XmlElement
+  public long getSize() {
+    return size;
+  }
+  
+  public void setSize(long size) {
+    this.size=size;
+  }
+
+  @XmlElement
+  public String getDate() {
+    return date;
+  }
+  
+  public void setDate(String date) {
+    this.date = date;    
+  }
+
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/rest/resource/ClientTrace.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/rest/resource/ClientTrace.java?rev=894540&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/rest/resource/ClientTrace.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/rest/resource/ClientTrace.java Wed Dec 30 04:30:23 2009
@@ -0,0 +1,111 @@
+/*
+ * 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.hadoop.chukwa.rest.resource;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import javax.ws.rs.DefaultValue;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.QueryParam;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.chukwa.Chunk;
+import org.apache.hadoop.chukwa.dataloader.SocketDataLoader;
+import org.apache.hadoop.chukwa.rest.bean.ClientTraceBean;
+import org.apache.hadoop.chukwa.rest.bean.EventBean;
+import org.apache.hadoop.chukwa.util.RecordConstants;
+
+/**
+ * Client Trace REST API for parsing client trace log file and convert
+ * data into consumable format for web browser and web services.
+ */
+@Path("clienttrace")
+public class ClientTrace {
+  protected static Log log = LogFactory.getLog(ClientTrace.class);
+  private static SocketDataLoader sdl = null;
+  // Client trace log file pattern
+  private final Pattern pattern =
+    Pattern.compile("(.+?) (.+?),(.+?) (.+?) src\\: /?(.+?):(.+?), dest\\: /?(.+?):(.+?), bytes\\: (\\d+), op\\: (.+?), cli(.+?)");
+
+  /**
+   * Get a list of the most recent client trace activities.
+   * The extracted elements are:
+   * 
+   * date   - Timestamp of the activity happened.
+   * action - Operation type: HDFS_READ, HDFS_WRITE, or MAPRED_SHUFFLE.
+   * src    - Source IP address
+   * dest   - Destination IP address
+   * size   - Size of the data payload.
+   * 
+   */
+  @GET
+  public List<ClientTraceBean> getTrace() {
+    if(sdl==null) {
+      sdl = new SocketDataLoader("ClientTrace");
+    } else if(!sdl.running()) {
+      sdl.start();
+    }
+
+    List<ClientTraceBean> list = new ArrayList<ClientTraceBean>();
+    try {
+      Collection<Chunk> clist = sdl.read();
+      for(Chunk c : clist) {
+        if(c!=null && c.getData()!=null) {
+          String action = "";
+          long size = 0;
+          String data = new String(c.getData());
+          String[] entries = data.split("\n");
+          for(String entry : entries) {
+            Matcher m = pattern.matcher(entry);
+            if(m.matches()) {
+              ClientTraceBean ctb = new ClientTraceBean();
+              size = Long.parseLong(m.group(9));
+              action = m.group(10);
+              StringBuilder date = new StringBuilder();
+              date.append(m.group(1));
+              date.append(" ");
+              date.append(m.group(2));
+              ctb.setDate(date.toString());
+              ctb.setSrc(m.group(5));
+              ctb.setDest(m.group(7));
+              ctb.setAction(action);
+              ctb.setSize(size);          
+              list.add(ctb);            
+            } else {
+              log.error("Unparsable line: "+entry);
+            }
+          }
+        }
+      }
+    } catch(NoSuchElementException e) {
+      log.debug("No data available for client trace.");
+    }
+    
+    return list;
+  }
+
+}

Added: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/dataloader/TestSocketDataLoader.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/dataloader/TestSocketDataLoader.java?rev=894540&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/dataloader/TestSocketDataLoader.java (added)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/dataloader/TestSocketDataLoader.java Wed Dec 30 04:30:23 2009
@@ -0,0 +1,70 @@
+/*
+ * 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.hadoop.chukwa.dataloader;
+
+import junit.framework.TestCase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.chukwa.ChunkImpl;
+import org.apache.hadoop.chukwa.Chunk;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.NoSuchElementException;
+import java.util.regex.Matcher;
+
+import org.apache.hadoop.chukwa.datacollection.collector.CaptureWriter;
+import org.apache.hadoop.chukwa.datacollection.writer.PipelineStageWriter;
+import org.apache.hadoop.chukwa.datacollection.writer.SocketTeeWriter;
+import org.apache.hadoop.chukwa.rest.bean.ClientTraceBean;
+
+import java.net.*;
+import java.io.*;
+
+public class TestSocketDataLoader  extends TestCase{
+  public void testSocketTee() throws Exception {
+    
+    Configuration conf = new Configuration();  
+    
+    conf.set("chukwaCollector.pipeline",
+        SocketTeeWriter.class.getCanonicalName());
+    
+    conf.set("chukwaCollector.writerClass", 
+        PipelineStageWriter.class.getCanonicalName());
+    
+    PipelineStageWriter psw = new PipelineStageWriter();
+    psw.init(conf);
+
+    SocketDataLoader sdl = new SocketDataLoader("all");
+    
+    System.out.println("pipeline established; now pushing a chunk");
+    ArrayList<Chunk> l = new ArrayList<Chunk>();
+    l.add(new ChunkImpl("dt", "name", 1, new byte[] {'a'}, null));
+    psw.add(l);
+    //push a chunk through. SocketDataLoader should receive this chunk.
+    
+    try {
+      Collection<Chunk> clist = sdl.read();
+      for(Chunk c : clist) {
+        if(c!=null && c.getData()!=null) {
+          assertEquals('a',c.getData());
+        }
+      }
+    } catch(NoSuchElementException e) {
+    }
+  }
+  
+}

Added: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/rest/resource/TestClientTrace.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/rest/resource/TestClientTrace.java?rev=894540&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/rest/resource/TestClientTrace.java (added)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/rest/resource/TestClientTrace.java Wed Dec 30 04:30:23 2009
@@ -0,0 +1,71 @@
+/*
+ * 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.hadoop.chukwa.rest.resource;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.chukwa.Chunk;
+import org.apache.hadoop.chukwa.ChunkImpl;
+import org.apache.hadoop.chukwa.datacollection.writer.PipelineStageWriter;
+import org.apache.hadoop.chukwa.datacollection.writer.SocketTeeWriter;
+import org.apache.hadoop.chukwa.datacollection.writer.WriterException;
+import org.apache.hadoop.chukwa.rest.bean.ClientTraceBean;
+import org.apache.hadoop.chukwa.rest.bean.UserBean;
+import org.apache.hadoop.chukwa.rest.bean.WidgetBean;
+import org.apache.hadoop.chukwa.util.ExceptionUtil;
+import org.apache.hadoop.conf.Configuration;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.GenericType;
+
+public class TestClientTrace extends SetupTestEnv {  
+  public void testClientTrace() {
+    // Setup Collector
+    Configuration conf = new Configuration();  
+    conf.set("chukwaCollector.pipeline",
+        SocketTeeWriter.class.getCanonicalName());
+    conf.set("chukwaCollector.writerClass", 
+        PipelineStageWriter.class.getCanonicalName());    
+    PipelineStageWriter psw = new PipelineStageWriter();
+    try {
+      psw.init(conf);
+      // Send a client trace chunk
+      ArrayList<Chunk> l = new ArrayList<Chunk>();
+      String line = "2009-12-29 22:32:27,047 INFO org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: /10.10.100.60:43707, dest: /10.10.100.60:50010, bytes: 7003141, op: HDFS_WRITE, cliID: DFSClient_-8389654, offset: 0, srvID: DS-2032680158-98.137.100.60-50010-1259976007324, blockid: blk_-2723720761101769540_705411, duration: 289013780000";      
+      l.add(new ChunkImpl("ClientTrace", "name", 1, line.getBytes(), null));
+      assertTrue(l.size()==1);
+      psw.add(l);
+      assertTrue(true);
+    } catch (WriterException e) {
+      fail(ExceptionUtil.getStackTrace(e));
+    }
+    
+    try {
+      // Locate the client trace object
+      client = Client.create();
+      resource = client.resource("http://localhost:"+restPort);
+      List<ClientTraceBean> list = resource.path("/hicc/v1/clienttrace").header("Authorization", authorization).get(new GenericType<List<ClientTraceBean>>(){});
+      for(ClientTraceBean ctb : list) {
+        assertEquals("HDFS_WRITE", ctb.getAction());
+      }
+    } catch (Exception e) {
+      fail(ExceptionUtil.getStackTrace(e));
+    }    
+  }
+}

Added: hadoop/chukwa/trunk/src/web/hicc/descriptors/hadoop_activity.descriptor
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/web/hicc/descriptors/hadoop_activity.descriptor?rev=894540&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/web/hicc/descriptors/hadoop_activity.descriptor (added)
+++ hadoop/chukwa/trunk/src/web/hicc/descriptors/hadoop_activity.descriptor Wed Dec 30 04:30:23 2009
@@ -0,0 +1,14 @@
+{
+"id":"hadoop_activity",
+"title":"Hadoop Activity",
+"version":"0.1",
+"categories":"Hadoop",
+"url":"iframe/jsp/hadoop_activity.jsp?height=650",
+"description":"Monitor clienttrace log file for Hadoop Activities",
+"screendump":"\/images\/start.png",
+"refresh":"15",
+"parameters":[
+{"name":"height","type":"string","value":"650","edit":"0"}
+]
+}
+

Added: hadoop/chukwa/trunk/src/web/hicc/js/activity.js
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/web/hicc/js/activity.js?rev=894540&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/web/hicc/js/activity.js (added)
+++ hadoop/chukwa/trunk/src/web/hicc/js/activity.js Wed Dec 30 04:30:23 2009
@@ -0,0 +1,206 @@
+/*
+ * 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.
+ */
+
+Array.prototype.remove = function(from, to) {
+  if (typeof from != "number") return this.remove(this.indexOf(from));
+  var rest = this.slice((to || from) + 1 || this.length);
+  this.length = from < 0 ? this.length + from : from;
+  return this.push.apply(this, rest);
+};
+
+jQuery(function ($) {
+  var el = document.getElementById("glow");
+  var frameRate = 60;
+  var speed = 5; // average # of comments per second
+  var width = el.width;
+  var height = el.height;
+  var p = Processing(el);
+  p.size(width, height);
+  p.colorMode(p.HSB);
+  p.noStroke();
+  var font = p.loadFont("Helvetica");
+  var fontSize = 12;
+  p.textFont(font, fontSize);
+  var components = [];
+  var bottom = 300;
+  
+  function whiteText(msg, x, y) {
+    p.textFont(font, fontSize);
+    p.fill(0, 0, 0);
+    p.rect(x, y - fontSize, font.width(msg) * fontSize, fontSize + 2);
+    p.fill(0, 0, 255);
+    p.text(msg, x, y);
+  }
+  
+  var Axes = function Axes() {
+    return {
+      update: function () {
+      },
+      draw: function () {
+        p.stroke(0, 0, 128);
+        p.line(15.5, 0, 15.5, height);
+        p.line(15, bottom + height / 10 + 0.5, width, bottom + height / 10 + 0.5);
+        for (var y = 0; y < height; y += height / 10) {
+          p.line(10, y + 0.5, 15, y + 0.5);
+        }
+        p.noStroke();
+        whiteText("overall activity", 20, fontSize * 1.5);
+        whiteText("hosts", width - font.width("hosts") * fontSize * 1.5, bottom + height / 10 + fontSize * 1.2);
+      }
+    }
+  };
+  components.push(new Axes());
+  
+  var Circle = function Circle(height, host, action) {
+    var maxAge = 100;
+    var letters = "0123456789".split("");
+    var x = 0;
+    var reverseIP = host.split("").reverse().join("");
+    for (var i=0; i<3; i++)
+      x += letters.indexOf(reverseIP.charAt(i).toUpperCase()) / Math.pow(letters.length, i + 1);
+    
+    return {
+      age: 0,
+      x: x * width * 4 / 5 + width / 5,
+      y: bottom - Math.sin(x * Math.PI) * Math.min(height, 90) * 3,
+      dx: (Math.random() * width - width / 2) * 0.001,
+      dy: (Math.random() * height - height / 2) * 0.001,
+      hue: Math.floor(x * 348),
+      update: function () {
+        this.age++;
+        this.x += this.dx;
+        this.y += this.dy;
+        if (this.age >= maxAge) {
+          components.remove(this);
+        }
+      },
+      draw: function () {
+        var age = this.age;
+        var hue = this.hue + age * 0.1;
+        var saturation = age + 155;
+        var brightness1 = 192 - age;
+        var brightness2 = 128 - age;
+        var alpha = 150 - 1.5 * age;
+        var r = age * age / 50 + 2 * age + 30; // parabola
+        p.fill(hue, saturation, brightness1, alpha);
+        p.ellipse(this.x, this.y, r, r);
+        p.fill(hue, saturation, brightness2, alpha);
+        p.ellipse(this.x, this.y, r * 4 / 5, r * 4 / 5);
+        p.fill(0, 0, 255, alpha);
+        p.textFont(font, fontSize);
+        var label = action + " " + host;
+        p.text(label, this.x - Math.floor(font.width(label) * fontSize / 2), this.y);
+      }
+    };
+  };
+  
+  function parseDate(isoDateString) {
+    var d = isoDateString.split(/[: -]/);
+    return new Date(Date.UTC(d[0], d[1] - 1, d[2], d[3], d[4], d[5]));
+  }
+  
+  function formatDate(date) {
+    var d = date.getDate();
+    d = "" + (d < 10 ? "0" : "") + d + " ";
+    d += "Jan Feb Mar Apr May Jun Jul Aug Sep Oct Nov Dec".split(/ /)[date.getMonth()] + " ";
+    d += date.getFullYear();
+    d = "Sun Mon Tue Wed Thu Fri Sat".split(/ /)[date.getDay()] + " " + d;
+    return d;
+  }
+  
+  function formatTime(date) {
+    var h = date.getHours();
+    var ampm = h >= 12 ? "PM" : "AM";
+    var t = "" + (h == 0 ? 12 : (h > 12 ? h - 12 : h)) + ":";
+    var m = date.getMinutes();
+    t += "" + (m < 10 ? "0" : "") + m + " " + ampm;
+    return t;
+  }
+  
+  var Glow = function Glow(comments, pages) {
+    var frames = comments.length * frameRate / speed;
+    var frame = 0;
+    return {
+      update: function () {
+        frame++;
+        while (comments.length > 0) {
+          var c = comments.shift();
+          if(c.date.getTime() >= last.getTime()) {
+            components.push(new Circle(components.length, c.src, c.action));
+          }
+        }
+        if (frame % 5 == 0)
+          $("#date").html(formatDate(last) + "<br/>" + formatTime(last));
+      },
+      draw: function () {
+        p.background(0, 15);
+      }
+    };
+  };
+  
+  var animate = function (frameRate, pause) {
+    var runner = function () {
+      for (var i=0; i<components.length; i++) {
+        components[i].update();
+      }
+      for (var i=0; i<components.length; i++) {
+        components[i].draw();
+      }
+    };
+    var interval = window.setInterval(runner, 1000 / frameRate);
+    $("#glow").toggle(function () {
+      window.clearInterval(interval);
+      p.textFont(font, 24);
+      p.fill(0, 0, 255);
+      p.text("Paused", width / 2 - font.width("Paused") * 12, 30);
+      runner(); // run once more to show message
+    }, function () {
+      interval = window.setInterval(runner, 1000 / frameRate);
+    });
+  };
+
+  var last; 
+  var refresh = setInterval(function () {
+    $.getJSON("/hicc/v1/clienttrace", function (data) {
+      var comments = [];
+      var pages = {};
+      if(data!=null) {
+        $.each(data.clientTraceBean, function (i, event) { 
+          var current = new Date();
+          comments.push({ 
+            id: current.getTime(), 
+            src: event.src,
+            action: event.action,
+            date: parseDate(event.date)
+            });
+          pages[i] = {
+            action: event.action,
+            date: parseDate(event.date),
+            src: event.src
+          };
+          last = parseDate(event.date);
+        }); 
+        components.push(new Glow(comments.reverse(), pages));
+      }
+
+    });
+  }, 15000);
+  animate(frameRate, $("#glow"));
+
+});
+

Added: hadoop/chukwa/trunk/src/web/hicc/js/processing.js
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/web/hicc/js/processing.js?rev=894540&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/web/hicc/js/processing.js (added)
+++ hadoop/chukwa/trunk/src/web/hicc/js/processing.js Wed Dec 30 04:30:23 2009
@@ -0,0 +1,1566 @@
+/*
+ * 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.
+ */
+/*
+ * Processing.js - John Resig (http://ejohn.org/)
+ * MIT Licensed
+ * http://ejohn.org/blog/processingjs/
+ *
+ * This is a port of the Processing Visualization Language.
+ * More information: http://processing.org/
+ */
+
+(function(){
+
+this.Processing = function Processing( aElement, aCode ) {
+  if ( typeof aElement == "string" )
+    aElement = document.getElementById( aElement );
+
+  var p = buildProcessing( aElement );
+
+  if ( aCode )
+    p.init( aCode );
+
+  return p;
+};
+
+function log() {
+  try {
+    console.log.apply( console, arguments );
+  } catch(e) {
+    try {
+      opera.postError.apply( opera, arguments );
+    } catch(e){}
+  }
+}
+
+var parse = Processing.parse = function parse( aCode, p ) {
+  // Angels weep at this parsing code :-(
+
+  // Remove end-of-line comments
+  aCode = aCode.replace(/\/\/ .*\n/g, "\n");
+
+  // Weird parsing errors with %
+  aCode = aCode.replace(/([^\s])%([^\s])/g, "$1 % $2");
+ 
+  // Simple convert a function-like thing to function
+  aCode = aCode.replace(/(?:static )?(\w+ )(\w+)\s*(\([^\)]*\)\s*{)/g, function(all, type, name, args) {
+    if ( name == "if" || name == "for" || name == "while" ) {
+      return all;
+    } else {
+      return "Processing." + name + " = function " + name + args;
+    }
+  });
+
+  // Force .length() to be .length
+  aCode = aCode.replace(/\.length\(\)/g, ".length");
+
+  // foo( int foo, float bar )
+  aCode = aCode.replace(/([\(,]\s*)(\w+)((?:\[\])+| )\s*(\w+\s*[\),])/g, "$1$4");
+  aCode = aCode.replace(/([\(,]\s*)(\w+)((?:\[\])+| )\s*(\w+\s*[\),])/g, "$1$4");
+
+  // float[] foo = new float[5];
+  aCode = aCode.replace(/new (\w+)((?:\[([^\]]*)\])+)/g, function(all, name, args) {
+    return "new ArrayList(" + args.slice(1,-1).split("][").join(", ") + ")";
+  });
+  
+  aCode = aCode.replace(/(?:static )?\w+\[\]\s*(\w+)\[?\]?\s*=\s*{.*?};/g, function(all) {
+    return all.replace(/{/g, "[").replace(/}/g, "]");
+  });
+
+  // int|float foo;
+  var intFloat = /(\n\s*(?:int|float)(?:\[\])?(?:\s*|[^\(]*?,\s*))([a-z]\w*)(;|,)/i;
+  while ( intFloat.test(aCode) ) {
+    aCode = aCode.replace(new RegExp(intFloat), function(all, type, name, sep) {
+      return type + " " + name + " = 0" + sep;
+    });
+  }
+
+  // float foo = 5;
+  aCode = aCode.replace(/(?:static )?(\w+)((?:\[\])+| ) *(\w+)\[?\]?(\s*[=,;])/g, function(all, type, arr, name, sep) {
+    if ( type == "return" )
+      return all;
+    else
+      return "var " + name + sep;
+  });
+
+  // Fix Array[] foo = {...} to [...]
+  aCode = aCode.replace(/=\s*{((.|\s)*?)};/g, function(all,data) {
+    return "= [" + data.replace(/{/g, "[").replace(/}/g, "]") + "]";
+  });
+  
+  // static { ... } blocks
+  aCode = aCode.replace(/static\s*{((.|\n)*?)}/g, function(all, init) {
+    // Convert the static definitons to variable assignments
+    //return init.replace(/\((.*?)\)/g, " = $1");
+    return init;
+  });
+
+  // super() is a reserved word
+  aCode = aCode.replace(/super\(/g, "superMethod(");
+
+  var classes = ["int", "float", "boolean", "string"];
+
+  function ClassReplace(all, name, extend, vars, last) {
+    classes.push( name );
+
+    var static = "";
+
+    vars = vars.replace(/final\s+var\s+(\w+\s*=\s*.*?;)/g, function(all,set) {
+      static += " " + name + "." + set;
+      return "";
+    });
+
+    // Move arguments up from constructor and wrap contents with
+    // a with(this), and unwrap constructor
+    return "function " + name + "() {with(this){\n  " +
+      (extend ? "var __self=this;function superMethod(){extendClass(__self,arguments," + extend + ");}\n" : "") +
+      // Replace var foo = 0; with this.foo = 0;
+      // and force var foo; to become this.foo = null;
+      vars
+        .replace(/,\s?/g, ";\n  this.")
+        .replace(/\b(var |final |public )+\s*/g, "this.")
+        .replace(/this.(\w+);/g, "this.$1 = null;") + 
+        (extend ? "extendClass(this, " + extend + ");\n" : "") +
+        "<CLASS " + name + " " + static + ">" + (typeof last == "string" ? last : name + "(");
+  }
+
+  var matchClasses = /(?:public |abstract |static )*class (\w+)\s*(?:extends\s*(\w+)\s*)?{\s*((?:.|\n)*?)\b\1\s*\(/g;
+  var matchNoCon = /(?:public |abstract |static )*class (\w+)\s*(?:extends\s*(\w+)\s*)?{\s*((?:.|\n)*?)(Processing)/g;
+  
+  aCode = aCode.replace(matchClasses, ClassReplace);
+  aCode = aCode.replace(matchNoCon, ClassReplace);
+
+  var matchClass = /<CLASS (\w+) (.*?)>/, m;
+  
+  while ( (m = aCode.match( matchClass )) ) {
+    var left = RegExp.leftContext,
+      allRest = RegExp.rightContext,
+      rest = nextBrace(allRest),
+      className = m[1],
+      staticVars = m[2] || "";
+      
+    allRest = allRest.slice( rest.length + 1 );
+
+    rest = rest.replace(new RegExp("\\b" + className + "\\(([^\\)]*?)\\)\\s*{", "g"), function(all, args) {
+      args = args.split(/,\s*?/);
+      
+      if ( args[0].match(/^\s*$/) )
+        args.shift();
+      
+      var fn = "if ( arguments.length == " + args.length + " ) {\n";
+        
+      for ( var i = 0; i < args.length; i++ ) {
+        fn += "    var " + args[i] + " = arguments[" + i + "];\n";
+      }
+        
+      return fn;
+    });
+    
+    // Fix class method names
+    // this.collide = function() { ... }
+    // and add closing } for with(this) ...
+    rest = rest.replace(/(?:public )?Processing.\w+ = function (\w+)\((.*?)\)/g, function(all, name, args) {
+      return "ADDMETHOD(this, '" + name + "', function(" + args + ")";
+    });
+    
+    var matchMethod = /ADDMETHOD([\s\S]*?{)/, mc;
+    var methods = "";
+    
+    while ( (mc = rest.match( matchMethod )) ) {
+      var prev = RegExp.leftContext,
+        allNext = RegExp.rightContext,
+        next = nextBrace(allNext);
+
+      methods += "addMethod" + mc[1] + next + "});"
+      
+      rest = prev + allNext.slice( next.length + 1 );
+    }
+
+    rest = methods + rest;
+    
+    aCode = left + rest + "\n}}" + staticVars + allRest;
+  }
+
+  // Do some tidying up, where necessary
+  aCode = aCode.replace(/Processing.\w+ = function addMethod/g, "addMethod");
+  
+  function nextBrace( right ) {
+    var rest = right;
+    var position = 0;
+    var leftCount = 1, rightCount = 0;
+    
+    while ( leftCount != rightCount ) {
+      var nextLeft = rest.indexOf("{");
+      var nextRight = rest.indexOf("}");
+      
+      if ( nextLeft < nextRight && nextLeft != -1 ) {
+        leftCount++;
+        rest = rest.slice( nextLeft + 1 );
+        position += nextLeft + 1;
+      } else {
+        rightCount++;
+        rest = rest.slice( nextRight + 1 );
+        position += nextRight + 1;
+      }
+    }
+    
+    return right.slice(0, position - 1);
+  }
+
+  // Handle (int) Casting
+  aCode = aCode.replace(/\(int\)/g, "0|");
+
+  // Remove Casting
+  aCode = aCode.replace(new RegExp("\\((" + classes.join("|") + ")(\\[\\])?\\)", "g"), "");
+  
+  // Convert 3.0f to just 3.0
+  aCode = aCode.replace(/(\d+)f/g, "$1");
+
+  // Force numbers to exist
+  //aCode = aCode.replace(/([^.])(\w+)\s*\+=/g, "$1$2 = ($2||0) +");
+
+  // Force characters-as-bytes to work
+  aCode = aCode.replace(/('[a-zA-Z0-9]')/g, "$1.charCodeAt(0)");
+
+  // Convert #aaaaaa into color
+  aCode = aCode.replace(/#([a-f0-9]{6})/ig, function(m, hex){
+    var num = toNumbers(hex);
+    return "color(" + num[0] + "," + num[1] + "," + num[2] + ")";
+  });
+
+  function toNumbers( str ){
+    var ret = [];
+     str.replace(/(..)/g, function(str){
+      ret.push( parseInt( str, 16 ) );
+    });
+    return ret;
+  }
+
+  log(aCode);
+
+  return aCode;
+};
+
+function buildProcessing( curElement ){
+
+  var p = {};
+
+  // init
+  p.PI = Math.PI;
+  p.TWO_PI = 2 * p.PI;
+  p.HALF_PI = p.PI / 2;
+  p.P3D = 3;
+  p.CORNER = 0;
+  p.RADIUS = 1;
+  p.CENTER_RADIUS = 1;
+  p.CENTER = 2;
+  p.POLYGON = 2;
+  p.QUADS = 5;
+  p.TRIANGLES = 6;
+  p.POINTS = 7;
+  p.LINES = 8;
+  p.TRIANGLE_STRIP = 9;
+  p.TRIANGLE_FAN = 4;
+  p.QUAD_STRIP = 3;
+  p.CORNERS = 10;
+  p.CLOSE = true;
+  p.RGB = 1;
+  p.HSB = 2;
+
+  // mouseButton constants: values adjusted to come directly from e.which
+  p.LEFT = 1;
+  p.CENTER = 2;
+  p.RIGHT = 3;
+
+  // "Private" variables used to maintain state
+  var curContext = curElement.getContext("2d");
+  var doFill = true;
+  var doStroke = true;
+  var loopStarted = false;
+  var hasBackground = false;
+  var doLoop = true;
+  var looping = 0;
+  var curRectMode = p.CORNER;
+  var curEllipseMode = p.CENTER;
+  var inSetup = false;
+  var inDraw = false;
+  var curBackground = "rgba(204,204,204,1)";
+  var curFrameRate = 1000;
+  var curShape = p.POLYGON;
+  var curShapeCount = 0;
+  var curvePoints = [];
+  var curTightness = 0;
+  var opacityRange = 255;
+  var redRange = 255;
+  var greenRange = 255;
+  var blueRange = 255;
+  var pathOpen = false;
+  var mousePressed = false;
+  var keyPressed = false;
+  var firstX, firstY, secondX, secondY, prevX, prevY;
+  var curColorMode = p.RGB;
+  var curTint = -1;
+  var curTextSize = 12;
+  var curTextFont = "Arial";
+  var getLoaded = false;
+  var start = (new Date).getTime();
+
+  // Global vars for tracking mouse position
+  p.pmouseX = 0;
+  p.pmouseY = 0;
+  p.mouseX = 0;
+  p.mouseY = 0;
+  p.mouseButton = 0;
+
+  // Will be replaced by the user, most likely
+  p.mouseDragged = undefined;
+  p.mouseMoved = undefined;
+  p.mousePressed = undefined;
+  p.mouseReleased = undefined;
+  p.keyPressed = undefined;
+  p.keyReleased = undefined;
+  p.draw = undefined;
+  p.setup = undefined;
+
+  // The height/width of the canvas
+  p.width = curElement.width - 0;
+  p.height = curElement.height - 0;
+
+  // The current animation frame
+  p.frameCount = 0;
+  
+  // In case I ever need to do HSV conversion:
+  // http://srufaculty.sru.edu/david.dailey/javascript/js/5rml.js
+  p.color = function color( aValue1, aValue2, aValue3, aValue4 ) {
+    var aColor = "";
+    
+    if ( arguments.length == 3 ) {
+      aColor = p.color( aValue1, aValue2, aValue3, opacityRange );
+    } else if ( arguments.length == 4 ) {
+      var a = aValue4 / opacityRange;
+      a = isNaN(a) ? 1 : a;
+
+      if ( curColorMode == p.HSB ) {
+        var rgb = HSBtoRGB(aValue1, aValue2, aValue3);
+        var r = rgb[0], g = rgb[1], b = rgb[2];
+      } else {
+        var r = getColor(aValue1, redRange);
+        var g = getColor(aValue2, greenRange);
+        var b = getColor(aValue3, blueRange);
+      }
+
+      aColor = "rgba(" + r + "," + g + "," + b + "," + a + ")";
+    } else if ( typeof aValue1 == "string" ) {
+      aColor = aValue1;
+
+      if ( arguments.length == 2 ) {
+        var c = aColor.split(",");
+        c[3] = (aValue2 / opacityRange) + ")";
+        aColor = c.join(",");
+      }
+    } else if ( arguments.length == 2 ) {
+      aColor = p.color( aValue1, aValue1, aValue1, aValue2 );
+    } else if ( typeof aValue1 == "number" ) {
+      aColor = p.color( aValue1, aValue1, aValue1, opacityRange );
+    } else {
+      aColor = p.color( redRange, greenRange, blueRange, opacityRange );
+    }
+
+    // HSB conversion function from Mootools, MIT Licensed
+    function HSBtoRGB(h, s, b) {
+      h = (h / redRange) * 360;
+      s = (s / greenRange) * 100;
+      b = (b / blueRange) * 100;
+      var hue = h % 360;
+      var f = hue % 60;
+      var br = Math.round(b / 100 * 255);
+      var p = Math.round((b * (100 - s)) / 10000 * 255);
+      var q = Math.round((b * (6000 - s * f)) / 600000 * 255);
+      var t = Math.round((b * (6000 - s * (60 - f))) / 600000 * 255);
+      switch (Math.floor(hue / 60)){
+        case 0: return [br, t, p];
+        case 1: return [q, br, p];
+        case 2: return [p, br, t];
+        case 3: return [p, q, br];
+        case 4: return [t, p, br];
+        case 5: return [br, p, q];
+      }
+    }
+
+    function getColor( aValue, range ) {
+      return Math.round(255 * (aValue / range));
+    }
+    
+    return aColor;
+  }
+
+  p.nf = function( num, pad ) {
+    var str = "" + num;
+    while ( pad - str.length )
+      str = "0" + str;
+    return str;
+  };
+
+  p.AniSprite = function( prefix, frames ) {
+    this.images = [];
+    this.pos = 0;
+
+    for ( var i = 0; i < frames; i++ ) {
+      this.images.push( prefix + p.nf( i, ("" + frames).length ) + ".gif" );
+    }
+
+    this.display = function( x, y ) {
+      p.image( this.images[ this.pos ], x, y );
+
+      if ( ++this.pos >= frames )
+        this.pos = 0;
+    };
+
+    this.getWidth = function() {
+      return getImage(this.images[0]).width;
+    };
+
+    this.getHeight = function() {
+      return getImage(this.images[0]).height;
+    };
+  };
+
+  function buildImageObject( obj ) {
+    var pixels = obj.data;
+    var data = p.createImage( obj.width, obj.height );
+
+    if ( data.__defineGetter__ && data.__lookupGetter__ && !data.__lookupGetter__("pixels") ) {
+      var pixelsDone;
+      data.__defineGetter__("pixels", function() {
+        if ( pixelsDone )
+          return pixelsDone;
+
+        pixelsDone = [];
+
+        for ( var i = 0; i < pixels.length; i += 4 ) {
+          pixelsDone.push( p.color(pixels[i], pixels[i+1], pixels[i+2], pixels[i+3]) );
+        }
+
+        return pixelsDone;
+      });
+    } else {
+      data.pixels = [];
+
+      for ( var i = 0; i < pixels.length; i += 4 ) {
+        data.pixels.push( p.color(pixels[i], pixels[i+1], pixels[i+2], pixels[i+3]) );
+      }
+    }
+
+    return data;
+  }
+
+  p.createImage = function createImage( w, h, mode ) {
+    var data = {};
+    data.width = w;
+    data.height = h;
+    data.data = [];
+
+    if ( curContext.createImageData ) {
+      data = curContext.createImageData( w, h );
+    }
+
+    data.pixels = new Array( w * h );
+    data.get = function(x,y) {
+      return this.pixels[w*y+x];
+    };
+    data._mask = null;
+    data.mask = function(img) {
+      this._mask = img;
+    };
+    data.loadPixels = function(){};
+    data.updatePixels = function(){};
+
+    return data;
+  };
+
+  p.createGraphics = function createGraphics( w, h ) {
+    var canvas = document.createElement("canvas");
+    var ret = buildProcessing( canvas );
+    ret.size( w, h );
+    ret.canvas = canvas;
+    return ret;
+  };
+
+  p.beginDraw = function beginDraw(){};
+
+  p.endDraw = function endDraw(){};
+
+  p.tint = function tint( rgb, a ) {
+    curTint = a;
+  };
+
+  function getImage( img ) {
+    if ( typeof img == "string" ) {
+      return document.getElementById(img);
+    }
+
+    if ( img.img || img.canvas ) {
+      return img.img || img.canvas;
+    }
+
+    for ( var i = 0, l = img.pixels.length; i < l; i++ ) {
+      var pos = i * 4;
+      var c = (img.pixels[i] || "rgba(0,0,0,1)").slice(5,-1).split(",");
+      img.data[pos] = parseInt(c[0]);
+      img.data[pos+1] = parseInt(c[1]);
+      img.data[pos+2] = parseInt(c[2]);
+      img.data[pos+3] = parseFloat(c[3]) * 100;
+    }
+
+    var canvas = document.createElement("canvas")
+    canvas.width = img.width;
+    canvas.height = img.height;
+    var context = canvas.getContext("2d");
+    context.putImageData( img, 0, 0 );
+
+    img.canvas = canvas;
+
+    return canvas;
+  }
+
+  p.image = function image( img, x, y, w, h ) {
+    x = x || 0;
+    y = y || 0;
+
+    var obj = getImage(img);
+
+    if ( curTint >= 0 ) {
+      var oldAlpha = curContext.globalAlpha;
+      curContext.globalAlpha = curTint / opacityRange;
+    }
+
+    if ( arguments.length == 3 ) {
+      curContext.drawImage( obj, x, y );
+    } else {
+      curContext.drawImage( obj, x, y, w, h );
+    }
+
+    if ( curTint >= 0 ) {
+      curContext.globalAlpha = oldAlpha;
+    }
+
+    if ( img._mask ) {
+      var oldComposite = curContext.globalCompositeOperation;
+      curContext.globalCompositeOperation = "darker";
+      p.image( img._mask, x, y );
+      curContext.globalCompositeOperation = oldComposite;
+    }
+  };
+
+  p.exit = function exit() {
+    clearInterval(looping);
+  };
+
+  p.save = function save( file ){};
+
+  p.loadImage = function loadImage( file ) {
+    var img = document.getElementById(file);
+    if ( !img )
+      return;
+
+    var h = img.height, w = img.width;
+
+    var canvas = document.createElement("canvas");
+    canvas.width = w;
+    canvas.height = h;
+    var context = canvas.getContext("2d");
+
+    context.drawImage( img, 0, 0 );
+    var data = buildImageObject( context.getImageData( 0, 0, w, h ) );
+    data.img = img;
+    return data;
+  };
+
+  p.loadFont = function loadFont( name ) {
+    return {
+      name: name,
+      width: function( str ) {
+        if ( curContext.mozMeasureText ) {
+          curContext.save();
+          curContext.mozTextStyle = curTextSize + "px " + curTextFont.name;
+          var width = curContext.mozMeasureText( typeof str == "number" ?
+            String.fromCharCode( str ) :
+            str) / curTextSize;
+          curContext.restore();
+          return width;
+        } else {
+          return 0;
+        }
+      }
+    };
+  };
+
+  p.textFont = function textFont( name, size ) {
+    curTextFont = name;
+    p.textSize( size );
+  };
+
+  p.textSize = function textSize( size ) {
+    if ( size ) {
+      curTextSize = size;
+    }
+  };
+
+  p.textAlign = function textAlign(){};
+
+  p.text = function text( str, x, y ) {
+    if ( str && curContext.mozDrawText ) {
+      curContext.save();
+      curContext.mozTextStyle = curTextSize + "px " + curTextFont.name;
+      curContext.translate(x, y);
+      curContext.mozDrawText( typeof str == "number" ?
+        String.fromCharCode( str ) :
+        str );
+      curContext.restore();
+    }
+  };
+
+  p.char = function char( key ) {
+    return key;
+  };
+
+  p.println = function println(){};
+
+  p.map = function map( value, istart, istop, ostart, ostop ) {
+    return ostart + (ostop - ostart) * ((value - istart) / (istop - istart));
+  };
+
+  String.prototype.replaceAll = function(re, replace) {
+    return this.replace(new RegExp(re, "g"), replace);
+  };
+
+  p.Point = function Point( x, y ) {
+    this.x = x;
+    this.y = y;
+    this.copy = function() {
+      return new Point( x, y );
+    }
+  };
+
+  p.Random = function() {
+    var haveNextNextGaussian = false;
+    var nextNextGaussian;
+
+    this.nextGaussian = function() {
+      if (haveNextNextGaussian) {
+        haveNextNextGaussian = false;
+
+        return nextNextGaussian;
+      } else {
+        var v1, v2, s;
+        do { 
+          v1 = 2 * p.random(1) - 1;   // between -1.0 and 1.0
+          v2 = 2 * p.random(1) - 1;   // between -1.0 and 1.0
+          s = v1 * v1 + v2 * v2;
+        } while (s >= 1 || s == 0);
+        var multiplier = Math.sqrt(-2 * Math.log(s)/s);
+        nextNextGaussian = v2 * multiplier;
+        haveNextNextGaussian = true;
+
+        return v1 * multiplier;
+      }
+    };
+  };
+
+  p.ArrayList = function ArrayList( size, size2, size3 ) {
+    var array = new Array( 0 | size );
+    
+    if ( size2 ) {
+      for ( var i = 0; i < size; i++ ) {
+        array[i] = [];
+
+        for ( var j = 0; j < size2; j++ ) {
+          var a = array[i][j] = size3 ? new Array( size3 ) : 0;
+          for ( var k = 0; k < size3; k++ ) {
+            a[k] = 0;
+          }
+        }
+      }
+    } else {
+      for ( var i = 0; i < size; i++ ) {
+        array[i] = 0;
+      }
+    }
+    
+    array.size = function() {
+      return this.length;
+    };
+    array.get = function( i ) {
+      return this[ i ];
+    };
+    array.remove = function( i ) {
+      return this.splice( i, 1 );
+    };
+    array.add = function( item ) {
+      return this.push( item );
+    };
+    array.clone = function() {
+      var a = new ArrayList( size );
+      for ( var i = 0; i < size; i++ ) {
+        a[ i ] = this[ i ];
+      }
+      return a;
+    };
+    array.isEmpty = function() {
+      return !this.length;
+    };
+    array.clear = function() {
+      this.length = 0;
+    };
+    
+    return array;
+  };
+  
+  p.colorMode = function colorMode( mode, range1, range2, range3, range4 ) {
+    curColorMode = mode;
+    
+    if ( arguments.length == 1) {
+      redRange = mode == p.HSB ? 360 : 255;
+    }
+
+    if ( arguments.length >= 4 ) {
+      redRange = range1;
+      greenRange = range2;
+      blueRange = range3;
+    }
+
+    if ( arguments.length == 5 ) {
+      opacityRange = range4;
+    }
+
+    if ( arguments.length == 2 ) {
+      p.colorMode( mode, range1, range1, range1, range1 );
+    }
+  };
+  
+  p.beginShape = function beginShape( type ) {
+    curShape = type;
+    curShapeCount = 0; 
+    curvePoints = [];
+  };
+  
+  p.endShape = function endShape( close ) {
+    if ( curShapeCount != 0 ) {
+      if ( close || doFill ) 
+      curContext.lineTo( firstX, firstY );
+
+      if ( doFill )
+        curContext.fill();
+        
+      if ( doStroke )
+        curContext.stroke();
+    
+      curContext.closePath();
+      curShapeCount = 0;
+      pathOpen = false;
+    }
+
+    if ( pathOpen ) {
+      if ( doFill )
+        curContext.fill();
+
+      if ( doStroke )
+        curContext.stroke();
+
+      curContext.closePath();
+      curShapeCount = 0;
+      pathOpen = false;
+    }
+  };
+  
+  p.vertex = function vertex( x, y, x2, y2, x3, y3 ) {
+    if ( curShapeCount == 0 && curShape != p.POINTS ) {
+      pathOpen = true;
+      curContext.beginPath();
+      curContext.moveTo( x, y );
+      firstX = x;
+      firstY = y;
+    } else {
+      if ( curShape == p.POINTS ) {
+        p.point( x, y );
+      } else if ( arguments.length == 2 ) {
+        if ( curShape != p.QUAD_STRIP || curShapeCount != 2 )
+          curContext.lineTo( x, y );
+
+        if ( curShape == p.TRIANGLE_STRIP ) {
+          if ( curShapeCount == 2 ) {
+            // finish shape
+            p.endShape(p.CLOSE);
+            pathOpen = true;
+            curContext.beginPath();
+            
+            // redraw last line to start next shape
+            curContext.moveTo( prevX, prevY );
+            curContext.lineTo( x, y );
+            curShapeCount = 1;
+          }
+          firstX = prevX;
+          firstY = prevY;
+        }
+
+        if ( curShape == p.TRIANGLE_FAN && curShapeCount == 2 ) {
+          // finish shape
+          p.endShape(p.CLOSE);
+          pathOpen = true;
+          curContext.beginPath();
+      
+          // redraw last line to start next shape
+          curContext.moveTo( firstX, firstY );
+          curContext.lineTo( x, y );
+          curShapeCount = 1;
+        }
+    
+        if ( curShape == p.QUAD_STRIP && curShapeCount == 3 ) {
+          // finish shape
+          curContext.lineTo( prevX, prevY );
+          p.endShape(p.CLOSE);
+          pathOpen = true;
+          curContext.beginPath();
+    
+          // redraw lines to start next shape
+          curContext.moveTo( prevX, prevY );
+          curContext.lineTo( x, y );
+          curShapeCount = 1;
+        }
+
+        if ( curShape == p.QUAD_STRIP) {
+          firstX = secondX;
+          firstY = secondY;
+          secondX = prevX;
+          secondY = prevY;
+        }
+      } else if ( arguments.length == 4 ) {
+        if ( curShapeCount > 1 ) {
+          curContext.moveTo( prevX, prevY );
+          curContext.quadraticCurveTo( firstX, firstY, x, y );
+          curShapeCount = 1;
+        }
+      } else if ( arguments.length == 6 ) {
+        curContext.bezierCurveTo( x, y, x2, y2, x3, y3 );
+        curShapeCount = -1;
+      }
+    }
+
+    prevX = x;
+    prevY = y;
+    curShapeCount++;
+    
+    if ( curShape == p.LINES && curShapeCount == 2 ||
+         (curShape == p.TRIANGLES) && curShapeCount == 3 ||
+     (curShape == p.QUADS) && curShapeCount == 4 ) {
+      p.endShape(p.CLOSE);
+    }
+  };
+
+  p.curveVertex = function( x, y, x2, y2 ) {
+    if ( curvePoints.length < 3 ) {
+      curvePoints.push([x,y]);
+    } else {
+      var b = [], s = 1 - curTightness;
+
+      /*
+       * Matrix to convert from Catmull-Rom to cubic Bezier
+       * where t = curTightness
+       * |0         1          0         0       |
+       * |(t-1)/6   1          (1-t)/6   0       |
+       * |0         (1-t)/6    1         (t-1)/6 |
+       * |0         0          0         0       |
+       */
+
+      curvePoints.push([x,y]);
+
+      b[0] = [curvePoints[1][0],curvePoints[1][1]];
+      b[1] = [curvePoints[1][0]+(s*curvePoints[2][0]-s*curvePoints[0][0])/6,curvePoints[1][1]+(s*curvePoints[2][1]-s*curvePoints[0][1])/6];
+      b[2] = [curvePoints[2][0]+(s*curvePoints[1][0]-s*curvePoints[3][0])/6,curvePoints[2][1]+(s*curvePoints[1][1]-s*curvePoints[3][1])/6];
+      b[3] = [curvePoints[2][0],curvePoints[2][1]];
+
+      if ( !pathOpen ) {
+        p.vertex( b[0][0], b[0][1] );
+      } else {
+        curShapeCount = 1;
+      }
+
+      p.vertex( b[1][0], b[1][1], b[2][0], b[2][1], b[3][0], b[3][1] );
+      curvePoints.shift();
+    }
+  };
+
+  p.curveTightness = function( tightness ) {
+    curTightness = tightness;
+  };
+
+  p.bezierVertex = p.vertex;
+  
+  p.rectMode = function rectMode( aRectMode ) {
+    curRectMode = aRectMode;
+  };
+
+  p.imageMode = function(){};
+  
+  p.ellipseMode = function ellipseMode( aEllipseMode ) {
+    curEllipseMode = aEllipseMode;
+  };
+  
+  p.dist = function dist( x1, y1, x2, y2 ) {
+    return Math.sqrt( Math.pow( x2 - x1, 2 ) + Math.pow( y2 - y1, 2 ) );
+  };
+
+  p.year = function year() {
+    return (new Date).getYear() + 1900;
+  };
+
+  p.month = function month() {
+    return (new Date).getMonth();
+  };
+
+  p.day = function day() {
+    return (new Date).getDay();
+  };
+
+  p.hour = function hour() {
+    return (new Date).getHours();
+  };
+
+  p.minute = function minute() {
+    return (new Date).getMinutes();
+  };
+
+  p.second = function second() {
+    return (new Date).getSeconds();
+  };
+
+  p.millis = function millis() {
+    return (new Date).getTime() - start;
+  };
+  
+  p.ortho = function ortho(){};
+  
+  p.translate = function translate( x, y ) {
+    curContext.translate( x, y );
+  };
+  
+  p.scale = function scale( x, y ) {
+    curContext.scale( x, y || x );
+  };
+  
+  p.rotate = function rotate( aAngle ) {
+    curContext.rotate( aAngle );
+  };
+  
+  p.pushMatrix = function pushMatrix() {
+    curContext.save();
+  };
+  
+  p.popMatrix = function popMatrix() {
+    curContext.restore();
+  };
+  
+  p.redraw = function redraw() {
+    if ( hasBackground ) {
+      p.background();
+    }
+
+    p.frameCount++;
+    
+    inDraw = true;
+    p.pushMatrix();
+    p.draw();
+    p.popMatrix();
+    inDraw = false;
+  };
+  
+  p.loop = function loop() {
+    if ( loopStarted )
+      return;
+    
+    looping = setInterval(function() {
+      try {
+        p.redraw();
+      }
+      catch(e) {
+        clearInterval( looping );
+        throw e;
+      }
+    }, 1000 / curFrameRate );
+    
+    loopStarted = true;
+  };
+  
+  p.frameRate = function frameRate( aRate ) {
+    curFrameRate = aRate;
+  };
+  
+  p.background = function background( img ) {
+    if ( arguments.length ) {
+      if ( img && img.img ) {
+        curBackground = img;
+      } else {
+        curBackground = p.color.apply( this, arguments );
+      }
+    }
+    
+
+    if ( curBackground.img ) {
+      p.image( curBackground, 0, 0 );
+    } else {
+      var oldFill = curContext.fillStyle;
+      curContext.fillStyle = curBackground + "";
+      curContext.fillRect( 0, 0, p.width, p.height );
+      curContext.fillStyle = oldFill;
+    }
+  };
+
+  p.sq = function sq( aNumber ) {
+    return aNumber * aNumber;
+  };
+
+  p.sqrt = function sqrt( aNumber ) {
+    return Math.sqrt( aNumber );
+  };
+  
+  p.int = function int( aNumber ) {
+    return Math.floor( aNumber );
+  };
+
+  p.min = function min( aNumber, aNumber2 ) {
+    return Math.min( aNumber, aNumber2 );
+  };
+
+  p.max = function max( aNumber, aNumber2 ) {
+    return Math.max( aNumber, aNumber2 );
+  };
+
+  p.ceil = function ceil( aNumber ) {
+    return Math.ceil( aNumber );
+  };
+
+  p.round = function round( aNumber ) {
+    return Math.round( aNumber );
+  };
+
+  p.floor = function floor( aNumber ) {
+    return Math.floor( aNumber );
+  };
+
+  p.float = function float( aNumber ) {
+    return typeof aNumber == "string" ?
+      p.float( aNumber.charCodeAt(0) ) :
+      parseFloat( aNumber );
+  };
+
+  p.byte = function byte( aNumber ) {
+    return aNumber || 0;
+  };
+  
+  p.random = function random( aMin, aMax ) {
+    return arguments.length == 2 ?
+      aMin + (Math.random() * (aMax - aMin)) :
+      Math.random() * aMin;
+  };
+
+  // From: http://freespace.virgin.net/hugo.elias/models/m_perlin.htm
+  p.noise = function( x, y, z ) {
+    return arguments.length >= 2 ?
+      PerlinNoise_2D( x, y ) :
+      PerlinNoise_2D( x, x );
+  };
+
+  function Noise(x, y) {
+    var n = x + y * 57;
+    n = (n<<13) ^ n;
+    return Math.abs(1.0 - (((n * ((n * n * 15731) + 789221) + 1376312589) & 0x7fffffff) / 1073741824.0));
+  };
+
+  function SmoothedNoise(x, y) {
+    var corners = ( Noise(x-1, y-1)+Noise(x+1, y-1)+Noise(x-1, y+1)+Noise(x+1, y+1) ) / 16;
+    var sides   = ( Noise(x-1, y)  +Noise(x+1, y)  +Noise(x, y-1)  +Noise(x, y+1) ) /  8;
+    var center  =  Noise(x, y) / 4;
+    return corners + sides + center;
+  };
+
+  function InterpolatedNoise(x, y) {
+    var integer_X    = Math.floor(x);
+    var fractional_X = x - integer_X;
+
+    var integer_Y    = Math.floor(y);
+    var fractional_Y = y - integer_Y;
+
+    var v1 = SmoothedNoise(integer_X,     integer_Y);
+    var v2 = SmoothedNoise(integer_X + 1, integer_Y);
+    var v3 = SmoothedNoise(integer_X,     integer_Y + 1);
+    var v4 = SmoothedNoise(integer_X + 1, integer_Y + 1);
+
+    var i1 = Interpolate(v1 , v2 , fractional_X);
+    var i2 = Interpolate(v3 , v4 , fractional_X);
+
+    return Interpolate(i1 , i2 , fractional_Y);
+  }
+
+  function PerlinNoise_2D(x, y) {
+      var total = 0;
+      var p = 0.25;
+      var n = 3;
+
+      for ( var i = 0; i <= n; i++ ) {
+          var frequency = Math.pow(2, i);
+          var amplitude = Math.pow(p, i);
+
+          total = total + InterpolatedNoise(x * frequency, y * frequency) * amplitude;
+      }
+
+      return total;
+  }
+
+  function Interpolate(a, b, x) {
+    var ft = x * p.PI;
+    var f = (1 - p.cos(ft)) * .5;
+    return  a*(1-f) + b*f;
+  }
+
+  p.red = function( aColor ) {
+    return parseInt(aColor.slice(5));
+  };
+
+  p.green = function( aColor ) {
+    return parseInt(aColor.split(",")[1]);
+  };
+
+  p.blue = function( aColor ) {
+    return parseInt(aColor.split(",")[2]);
+  };
+
+  p.alpha = function( aColor ) {
+    return parseInt(aColor.split(",")[3]);
+  };
+
+  p.abs = function abs( aNumber ) {
+    return Math.abs( aNumber );
+  };
+  
+  p.cos = function cos( aNumber ) {
+    return Math.cos( aNumber );
+  };
+  
+  p.sin = function sin( aNumber ) {
+    return Math.sin( aNumber );
+  };
+  
+  p.pow = function pow( aNumber, aExponent ) {
+    return Math.pow( aNumber, aExponent );
+  };
+  
+  p.constrain = function constrain( aNumber, aMin, aMax ) {
+    return Math.min( Math.max( aNumber, aMin ), aMax );
+  };
+  
+  p.sqrt = function sqrt( aNumber ) {
+    return Math.sqrt( aNumber );
+  };
+  
+  p.atan2 = function atan2( aNumber, aNumber2 ) {
+    return Math.atan2( aNumber, aNumber2 );
+  };
+  
+  p.radians = function radians( aAngle ) {
+    return ( aAngle / 180 ) * p.PI;
+  };
+  
+  p.size = function size( aWidth, aHeight ) {
+    var fillStyle = curContext.fillStyle;
+    var strokeStyle = curContext.strokeStyle;
+
+    curElement.width = p.width = aWidth;
+    curElement.height = p.height = aHeight;
+
+    curContext.fillStyle = fillStyle;
+    curContext.strokeStyle = strokeStyle;
+  };
+  
+  p.noStroke = function noStroke() {
+    doStroke = false;
+  };
+  
+  p.noFill = function noFill() {
+    doFill = false;
+  };
+  
+  p.smooth = function smooth(){};
+  
+  p.noLoop = function noLoop() {
+    doLoop = false;
+  };
+  
+  p.fill = function fill() {
+    doFill = true;
+    curContext.fillStyle = p.color.apply( this, arguments );
+  };
+  
+  p.stroke = function stroke() {
+    doStroke = true;
+    curContext.strokeStyle = p.color.apply( this, arguments );
+  };
+
+  p.strokeWeight = function strokeWeight( w ) {
+    curContext.lineWidth = w;
+  };
+  
+  p.point = function point( x, y ) {
+    var oldFill = curContext.fillStyle;
+    curContext.fillStyle = curContext.strokeStyle;
+    curContext.fillRect( Math.round( x ), Math.round( y ), 1, 1 );
+    curContext.fillStyle = oldFill;
+  };
+
+  p.get = function get( x, y ) {
+    if ( arguments.length == 0 ) {
+      var c = p.createGraphics( p.width, p.height );
+      c.image( curContext, 0, 0 );
+      return c;
+    }
+
+    if ( !getLoaded ) {
+      getLoaded = buildImageObject( curContext.getImageData(0, 0, p.width, p.height) );
+    }
+
+    return getLoaded.get( x, y );
+  };
+
+  p.set = function set( x, y, obj ) {
+    if ( obj && obj.img ) {
+      p.image( obj, x, y );
+    } else {
+      var oldFill = curContext.fillStyle;
+      var color = obj;
+      curContext.fillStyle = color;
+      curContext.fillRect( Math.round( x ), Math.round( y ), 1, 1 );
+      curContext.fillStyle = oldFill;
+    }
+  };
+  
+  p.arc = function arc( x, y, width, height, start, stop ) {
+    if ( width <= 0 )
+      return;
+
+    if ( curEllipseMode == p.CORNER ) {
+      x += width / 2;
+      y += height / 2;
+    }
+
+    curContext.beginPath();
+  
+    curContext.moveTo( x, y );
+    curContext.arc( x, y, curEllipseMode == p.CENTER_RADIUS ? width : width/2, start, stop, false );
+    
+    if ( doFill )
+      curContext.fill();
+      
+    if ( doStroke )
+      curContext.stroke();
+    
+    curContext.closePath();
+  };
+  
+  p.line = function line( x1, y1, x2, y2 ) {
+    curContext.lineCap = "round";
+    curContext.beginPath();
+  
+    curContext.moveTo( x1 || 0, y1 || 0 );
+    curContext.lineTo( x2 || 0, y2 || 0 );
+    
+    curContext.stroke();
+    
+    curContext.closePath();
+  };
+
+  p.bezier = function bezier( x1, y1, x2, y2, x3, y3, x4, y4 ) {
+    curContext.lineCap = "butt";
+    curContext.beginPath();
+  
+    curContext.moveTo( x1, y1 );
+    curContext.bezierCurveTo( x2, y2, x3, y3, x4, y4 );
+    
+    curContext.stroke();
+    
+    curContext.closePath();
+  };
+
+  p.triangle = function triangle( x1, y1, x2, y2, x3, y3 ) {
+    p.beginShape();
+    p.vertex( x1, y1 );
+    p.vertex( x2, y2 );
+    p.vertex( x3, y3 );
+    p.endShape();
+  };
+
+  p.quad = function quad( x1, y1, x2, y2, x3, y3, x4, y4 ) {
+    p.beginShape();
+    p.vertex( x1, y1 );
+    p.vertex( x2, y2 );
+    p.vertex( x3, y3 );
+    p.vertex( x4, y4 );
+    p.endShape();
+  };
+  
+  p.rect = function rect( x, y, width, height ) {
+    if ( width == 0 && height == 0 )
+      return;
+
+    curContext.beginPath();
+    
+    var offsetStart = 0;
+    var offsetEnd = 0;
+
+    if ( curRectMode == p.CORNERS ) {
+      width -= x;
+      height -= y;
+    }
+    
+    if ( curRectMode == p.RADIUS ) {
+      width *= 2;
+      height *= 2;
+    }
+    
+    if ( curRectMode == p.CENTER || curRectMode == p.RADIUS ) {
+      x -= width / 2;
+      y -= height / 2;
+    }
+  
+    curContext.rect(
+      Math.round( x ) - offsetStart,
+      Math.round( y ) - offsetStart,
+      Math.round( width ) + offsetEnd,
+      Math.round( height ) + offsetEnd
+    );
+      
+    if ( doFill )
+      curContext.fill();
+      
+    if ( doStroke )
+      curContext.stroke();
+    
+    curContext.closePath();
+  };
+  
+  p.ellipse = function ellipse( x, y, width, height ) {
+    x = x || 0;
+    y = y || 0;
+
+    if ( width <= 0 && height <= 0 )
+      return;
+
+    curContext.beginPath();
+    
+    if ( curEllipseMode == p.RADIUS ) {
+      width *= 2;
+      height *= 2;
+    }
+    
+    var offsetStart = 0;
+    
+    // Shortcut for drawing a circle
+    if ( width == height )
+      curContext.arc( x - offsetStart, y - offsetStart, width / 2, 0, Math.PI * 2, false );
+  
+    if ( doFill )
+      curContext.fill();
+      
+    if ( doStroke )
+      curContext.stroke();
+    
+    curContext.closePath();
+  };
+
+  p.link = function( href, target ) {
+    window.location = href;
+  };
+
+  p.loadPixels = function() {
+    p.pixels = buildImageObject( curContext.getImageData(0, 0, p.width, p.height) ).pixels;
+  };
+
+  p.updatePixels = function() {
+    var colors = /(\d+),(\d+),(\d+),(\d+)/;
+    var pixels = {};
+    pixels.width = p.width;
+    pixels.height = p.height;
+    pixels.data = [];
+
+    if ( curContext.createImageData ) {
+      pixels = curContext.createImageData( p.width, p.height );
+    }
+
+    var data = pixels.data;
+    var pos = 0;
+
+    for ( var i = 0, l = p.pixels.length; i < l; i++ ) {
+      var c = (p.pixels[i] || "rgba(0,0,0,1)").match(colors);
+      data[pos] = parseInt(c[1]);
+      data[pos+1] = parseInt(c[2]);
+      data[pos+2] = parseInt(c[3]);
+      data[pos+3] = parseFloat(c[4]) * 100;
+      pos += 4;
+    }
+
+    curContext.putImageData(pixels, 0, 0);
+  };
+
+  p.extendClass = function extendClass( obj, args, fn ) {
+    if ( arguments.length == 3 ) {
+      fn.apply( obj, args );
+    } else {
+      args.call( obj );
+    }
+  };
+
+  p.addMethod = function addMethod( object, name, fn ) {
+    if ( object[ name ] ) {
+      var args = fn.length;
+      
+      var oldfn = object[ name ];
+      object[ name ] = function() {
+        if ( arguments.length == args )
+          return fn.apply( this, arguments );
+        else
+          return oldfn.apply( this, arguments );
+      };
+    } else {
+      object[ name ] = fn;
+    }
+  };
+
+  p.init = function init(code){
+    p.stroke( 0 );
+    p.fill( 255 );
+  
+    // Canvas has trouble rendering single pixel stuff on whole-pixel
+    // counts, so we slightly offset it (this is super lame).
+    curContext.translate( 0.5, 0.5 );
+
+    if ( code ) {
+      (function(Processing){with (p){
+        eval(parse(code, p));
+      }})(p);
+    }
+  
+    if ( p.setup ) {
+      inSetup = true;
+      p.setup();
+    }
+    
+    inSetup = false;
+    
+    if ( p.draw ) {
+      if ( !doLoop ) {
+        p.redraw();
+      } else {
+        p.loop();
+      }
+    }
+    
+    attach( curElement, "mousemove", function(e) {
+      var scrollX = window.scrollX != null ? window.scrollX : window.pageXOffset;
+      var scrollY = window.scrollY != null ? window.scrollY : window.pageYOffset;
+      p.pmouseX = p.mouseX;
+      p.pmouseY = p.mouseY;
+      p.mouseX = e.clientX - curElement.offsetLeft + scrollX;
+      p.mouseY = e.clientY - curElement.offsetTop + scrollY;
+
+      if ( p.mouseMoved ) {
+        p.mouseMoved();
+      }      
+
+      if ( mousePressed && p.mouseDragged ) {
+        p.mouseDragged();
+      }      
+    });
+    
+    attach( curElement, "mousedown", function(e) {
+      mousePressed = true;
+      p.mouseButton = e.which;
+
+      if ( typeof p.mousePressed == "function" ) {
+        p.mousePressed();
+      } else {
+        p.mousePressed = true;
+      }
+    });
+
+    attach( curElement, "contextmenu", function(e) {
+      e.preventDefault();
+      e.stopPropagation();
+    });
+
+    attach( curElement, "mouseup", function(e) {
+      mousePressed = false;
+
+      if ( typeof p.mousePressed != "function" ) {
+        p.mousePressed = false;
+      }
+
+      if ( p.mouseReleased ) {
+        p.mouseReleased();
+      }
+    });
+
+    attach( document, "keydown", function(e) {
+      keyPressed = true;
+
+      p.key = e.keyCode + 32;
+
+      if ( e.shiftKey ) {
+        p.key = String.fromCharCode(p.key).toUpperCase().charCodeAt(0);
+      }
+
+      if ( typeof p.keyPressed == "function" ) {
+        p.keyPressed();
+      } else {
+        p.keyPressed = true;
+      }
+    });
+
+    attach( document, "keyup", function(e) {
+      keyPressed = false;
+
+      if ( typeof p.keyPressed != "function" ) {
+        p.keyPressed = false;
+      }
+
+      if ( p.keyReleased ) {
+        p.keyReleased();
+      }
+    });
+
+    function attach(elem, type, fn) {
+      if ( elem.addEventListener )
+        elem.addEventListener( type, fn, false );
+      else
+        elem.attachEvent( "on" + type, fn );
+    }
+  };
+
+  return p;
+}
+
+})();
+
+

Added: hadoop/chukwa/trunk/src/web/hicc/jsp/hadoop_activity.jsp
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/web/hicc/jsp/hadoop_activity.jsp?rev=894540&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/web/hicc/jsp/hadoop_activity.jsp (added)
+++ hadoop/chukwa/trunk/src/web/hicc/jsp/hadoop_activity.jsp Wed Dec 30 04:30:23 2009
@@ -0,0 +1,71 @@
+<%
+/*
+ * 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.
+ */
+%>
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Strict//EN"
+  "http://www.w3.org/TR/xhtml1/DTD/xhtml1-strict.dtd">
+
+<html xmlns="http://www.w3.org/1999/xhtml" xml:lang="en" lang="en">
+<head>
+  <meta http-equiv="Content-Type" content="text/html; charset=utf-8"/>
+  <title>Activity</title>
+  <script src="/hicc/js/processing.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/hicc/js/jquery-1.3.2.min.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/hicc/js/activity.js" type="text/javascript" charset="utf-8"></script>
+
+  <style type="text/css" media="screen">
+    body {
+        background: #000;
+        color: #aaa;
+        font-family: Helvetica, sans-serif;
+    }
+    h1 {
+        font-size: 2.5em;
+        margin: 0;
+        margin-top: .5em;
+        text-align: center;
+        color: #eee;
+        font-weight: bold;
+    }
+    #glow {
+      margin: 1em auto;
+      width: 1000px;
+      display: block;
+    }
+    #date {
+      text-align: center;
+      font-size: 1.5em;
+    }
+    #footer {
+        position: absolute;
+        bottom: 1em;
+        right: 1em;
+        font-size: 12px;
+    }
+    #footer a {
+        color: #eee;
+    }
+  </style>
+</head>
+<body>
+<h1>Hadoop activity</h1>
+<canvas id="glow" width="1000" height="500"></canvas>
+<p id="date"></p>
+</body>
+</html>
+