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 2010/10/17 20:20:33 UTC

svn commit: r1023544 - in /incubator/chukwa/trunk: ./ src/java/org/apache/hadoop/chukwa/datastore/ src/java/org/apache/hadoop/chukwa/hicc/ src/java/org/apache/hadoop/chukwa/hicc/bean/ src/java/org/apache/hadoop/chukwa/hicc/rest/ src/web/hicc/js/ src/we...

Author: eyang
Date: Sun Oct 17 18:20:33 2010
New Revision: 1023544

URL: http://svn.apache.org/viewvc?rev=1023544&view=rev
Log:
CHUKWA-520. Implemented graphing rest api and graph explorer for plotting HBase data to graph.  (Eric Yang)


Added:
    incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datastore/ChukwaHBaseStore.java
    incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/bean/
    incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/bean/Series.java
    incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/rest/
    incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/rest/MetricsController.java
    incubator/chukwa/trunk/src/web/hicc/js/autoHeight.js
    incubator/chukwa/trunk/src/web/hicc/jsp/graph_explorer.jsp
Modified:
    incubator/chukwa/trunk/CHANGES.txt
    incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/Chart.java
    incubator/chukwa/trunk/src/web/hicc/jsp/chart.jsp
    incubator/chukwa/trunk/src/web/hicc/jsp/host_selector_dropdown.jsp

Modified: incubator/chukwa/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/chukwa/trunk/CHANGES.txt?rev=1023544&r1=1023543&r2=1023544&view=diff
==============================================================================
--- incubator/chukwa/trunk/CHANGES.txt (original)
+++ incubator/chukwa/trunk/CHANGES.txt Sun Oct 17 18:20:33 2010
@@ -4,6 +4,8 @@ Trunk (unreleased changes)
 
   NEW FEATURES
 
+    CHUKWA-520. Implemented graphing rest api and graph explorer for plotting HBase data to graph.  (Eric Yang)
+
     CHUKWA-530. Implemented Syslog Adaptor for mappig Syslog facility name to Chukwa data type.  (Eric Yang)
 
     CHUKWA-419. Replaced system metrics collection with Sigar. (Eric Yang)

Added: incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datastore/ChukwaHBaseStore.java
URL: http://svn.apache.org/viewvc/incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datastore/ChukwaHBaseStore.java?rev=1023544&view=auto
==============================================================================
--- incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datastore/ChukwaHBaseStore.java (added)
+++ incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datastore/ChukwaHBaseStore.java Sun Oct 17 18:20:33 2010
@@ -0,0 +1,164 @@
+package org.apache.hadoop.chukwa.datastore;
+
+import java.util.ArrayList;
+import java.util.Calendar;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+
+import org.apache.hadoop.chukwa.datacollection.writer.hbase.HBaseWriter;
+import org.apache.hadoop.chukwa.hicc.bean.Series;
+import org.apache.hadoop.chukwa.util.ExceptionUtil;
+
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTablePool;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.RowFilter;
+import org.apache.hadoop.hbase.filter.SubstringComparator;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.log4j.Logger;
+import org.mortbay.log.Log;
+
+public class ChukwaHBaseStore {
+  private static HBaseConfiguration hconf = hconf = new HBaseConfiguration();
+  private static HTablePool pool = new HTablePool(hconf, 60);
+  static Logger log = Logger.getLogger(ChukwaHBaseStore.class);
+  
+  public static Series getSeries(String tableName, String rkey, String column,
+      long startTime, long endTime, boolean filterByRowKey) {
+    StringBuilder seriesName = new StringBuilder();
+    seriesName.append(rkey);
+    seriesName.append(":");
+    seriesName.append(column);
+    Series series = new Series(seriesName.toString());
+    try {
+      HTable table = pool.getTable(tableName);
+      Calendar c = Calendar.getInstance();
+      c.setTimeInMillis(startTime);
+      c.set(Calendar.MINUTE, 0);
+      c.set(Calendar.SECOND, 0);
+      c.set(Calendar.MILLISECOND, 0);
+      String startRow = c.getTimeInMillis()+rkey;
+      Scan scan = new Scan();
+      scan.addColumn(column.getBytes());
+      scan.setStartRow(startRow.getBytes());
+      scan.setTimeRange(startTime, endTime);
+      scan.setMaxVersions();
+      if(filterByRowKey) {
+        RowFilter rf = new RowFilter(CompareOp.EQUAL, new 
+            SubstringComparator(rkey)); 
+        scan.setFilter(rf);
+      }
+      ResultScanner results = table.getScanner(scan);
+      long step = startTime;
+      Iterator<Result> it = results.iterator();
+      // TODO: Apply discrete wavelet transformation to limit the output
+      // size to 1000 data points for graphing optimization. (i.e jwave)
+      while(it.hasNext()) {
+        Result result = it.next();
+        String temp = new String(result.getCellValue().getValue());
+        double value = Double.parseDouble(temp);
+        series.add(result.getCellValue().getTimestamp(), value);
+      }
+      results.close();
+      table.close();
+    } catch(Exception e) {
+      log.error(ExceptionUtil.getStackTrace(e));
+    }
+    return series;
+  }
+
+  public static Set<String> getFamilyNames(String tableName) {
+    Set<String> familyNames = new CopyOnWriteArraySet<String>();
+    try {
+      HTable table = pool.getTable(tableName);
+      Calendar c = Calendar.getInstance();
+      Set<byte[]> families = table.getTableDescriptor().getFamiliesKeys();
+      for(byte[] name : families) {
+        familyNames.add(new String(name));
+      }
+      table.close();
+    } catch(Exception e) {
+      log.error(ExceptionUtil.getStackTrace(e));
+    }
+    return familyNames;
+    
+  }
+  
+  public static Set<String> getTableNames() {
+    Set<String> tableNames = new CopyOnWriteArraySet<String>();
+    try {
+      HBaseAdmin admin = new HBaseAdmin(hconf);
+      HTableDescriptor[] td = admin.listTables();
+      for(HTableDescriptor table : td) {
+        tableNames.add(new String(table.getName()));
+      }
+    } catch(Exception e) {
+      log.error(ExceptionUtil.getStackTrace(e));
+    }
+    return tableNames;
+  }
+  
+  public static Set<String> getColumnNames(String tableName, String family, long startTime, long endTime) {
+    Set<String> columnNames = new CopyOnWriteArraySet<String>();
+    try {
+      HTable table = pool.getTable(tableName);
+      Scan scan = new Scan();
+      scan.setTimeRange(startTime, endTime);
+      scan.addFamily(family.getBytes());
+      ResultScanner results = table.getScanner(scan);
+      Iterator<Result> it = results.iterator();
+      while(it.hasNext()) {
+        Result result = it.next();
+        List<KeyValue> kvList = result.list();
+        for(KeyValue kv : kvList) {
+          columnNames.add(new String(kv.getColumn()));
+        }
+      }
+      results.close();
+      table.close();
+    } catch(Exception e) {
+      log.error(ExceptionUtil.getStackTrace(e));
+    }
+    return columnNames;
+  }
+  
+  public static Set<String> getRowNames(String tableName, String column, long startTime, long endTime) {
+    Set<String> rows = new HashSet<String>();
+    HTable table = pool.getTable(tableName);
+    try {
+      Scan scan = new Scan();
+      scan.addColumn(column.getBytes());
+      scan.setTimeRange(startTime, endTime);
+      ResultScanner results = table.getScanner(scan);
+      Iterator<Result> it = results.iterator();
+      while(it.hasNext()) {
+        Result result = it.next();
+        String buffer = new String(result.getRow());
+        String[] parts = buffer.split("-", 2);
+        if(!rows.contains(parts[1])) {
+          rows.add(parts[1]);
+        }
+      }
+      results.close();
+      table.close();
+    } catch(Exception e) {
+      log.error(ExceptionUtil.getStackTrace(e));
+    }
+    return rows;    
+  }
+  
+  public static Set<String> getHostnames(String cluster, long startTime, long endTime) {
+    return getRowNames("SystemMetrics","system:csource", startTime, endTime);
+  }
+}

Modified: incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/Chart.java
URL: http://svn.apache.org/viewvc/incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/Chart.java?rev=1023544&r1=1023543&r2=1023544&view=diff
==============================================================================
--- incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/Chart.java (original)
+++ incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/Chart.java Sun Oct 17 18:20:33 2010
@@ -33,6 +33,7 @@ import java.text.SimpleDateFormat;
 import javax.servlet.http.HttpServletRequest;
 import org.apache.hadoop.chukwa.hicc.ColorPicker;
 import org.apache.hadoop.chukwa.util.XssFilter;
+import org.json.JSONArray;
 
 @SuppressWarnings("unused")
 public class Chart {
@@ -454,31 +455,26 @@ public class Chart {
     }
     output.append(" ];\n");
     if(this.restData!=null) {
-        output.append("var _rest = [");
-        boolean first=true;
-        for(String url : restData) {
-            if(!first) {
-                output.append(",");
-            }
-            output.append("\"");
-            output.append(url);
-            output.append("\"");
-            first=false;
-        }
-        output.append("];");
+      JSONArray arr = new JSONArray();
+      for(String url : restData) {
+        arr.put(url);
+      }
+      output.append("var _rest = ");
+      output.append(arr.toString());
+      output.append(";");
     }
     if (request != null && xf.getParameter("format") == null) {
-	output.append("$(document).ready(function() { \n");
-        if(this.restData!=null) {
-            output.append("   loadData();\n");
-        } else {
-	    output.append("   wholePeriod();\n");
-        }
-	output.append("   $(window).resize(function() { wholePeriod(); });\n");
-	output.append("});\n");
-	output.append("</script>\n");
-        output.append("<input type=\"button\" value=\"Export\" onclick=\"javascript:saveReport();\">\n");
-	output.append("</body></html>\n");
+      output.append("$(document).ready(function() { \n");
+      if(this.restData!=null) {
+        output.append("   loadData();\n");
+      } else {
+        output.append("   wholePeriod();\n");
+      }
+      output.append("   $(window).resize(function() { wholePeriod(); });\n");
+      output.append("});\n");
+      output.append("</script>\n");
+      output.append("<input type=\"button\" value=\"Export\" onclick=\"javascript:saveReport();\">\n");
+      output.append("</body></html>\n");
     } else {
       output.append("chartTitle=\"<center>" + this.title + "</center>\";");
       output.append("height=" + this.height + ";");

Added: incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/bean/Series.java
URL: http://svn.apache.org/viewvc/incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/bean/Series.java?rev=1023544&view=auto
==============================================================================
--- incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/bean/Series.java (added)
+++ incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/bean/Series.java Sun Oct 17 18:20:33 2010
@@ -0,0 +1,57 @@
+package org.apache.hadoop.chukwa.hicc.bean;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+import javax.xml.bind.annotation.XmlValue;
+
+import org.apache.hadoop.chukwa.datastore.ChukwaHBaseStore;
+import org.apache.hadoop.chukwa.util.ExceptionUtil;
+import org.apache.log4j.Logger;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(propOrder={})
+public class Series {
+  private JSONObject series;
+  static Logger log = Logger.getLogger(Series.class);
+  
+  public Series(String name) {
+    series = new JSONObject();
+    try {
+      series.put("name", name);
+    } catch (JSONException e) {
+      log.error(ExceptionUtil.getStackTrace(e));
+    }
+  }
+  
+  public void add(long x, double y) {
+    try {
+    if(!series.has("data")) {
+      series.put("data", new JSONArray());
+    }
+    JSONArray xy = new JSONArray();
+    xy.put(x);
+    xy.put(y);
+    series.getJSONArray("data").put(xy);
+    } catch(Exception e) {
+      log.error(ExceptionUtil.getStackTrace(e));
+    }
+  }
+  
+  public String toString() {
+    return series.toString();
+  }
+
+  public Object toJSONObject() {
+    return series;
+  }
+}

Added: incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/rest/MetricsController.java
URL: http://svn.apache.org/viewvc/incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/rest/MetricsController.java?rev=1023544&view=auto
==============================================================================
--- incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/rest/MetricsController.java (added)
+++ incubator/chukwa/trunk/src/java/org/apache/hadoop/chukwa/hicc/rest/MetricsController.java Sun Oct 17 18:20:33 2010
@@ -0,0 +1,195 @@
+package org.apache.hadoop.chukwa.hicc.rest;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Set;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpSession;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.chukwa.datacollection.adaptor.sigar.SigarRunner;
+import org.apache.hadoop.chukwa.datastore.ChukwaHBaseStore;
+import org.apache.hadoop.chukwa.hicc.TimeHandler;
+import org.apache.hadoop.chukwa.hicc.bean.Series;
+import org.apache.log4j.Logger;
+import org.json.JSONArray;
+
+@Path("/metrics")
+public class MetricsController {
+  private static Logger log = Logger.getLogger(MetricsController.class);
+
+  @GET
+  @Path("series/{table}/{column}/rowkey/{rkey}")
+  @Produces("application/json")
+  public String getSeries(@Context HttpServletRequest request, @PathParam("table") String table, @PathParam("column") String column, @PathParam("rkey") String rkey, @QueryParam("start") String start, @QueryParam("end") String end) {
+    SimpleDateFormat sdf = new SimpleDateFormat("yyyyMMddHHmmss");
+    String buffer = "";
+    Series series;
+    long startTime = 0;
+    long endTime = 0;
+    TimeHandler time = new TimeHandler(request);
+    try {
+      if(start!=null) {
+        startTime = sdf.parse(start).getTime();
+      } else {
+        startTime = time.getStartTime();
+      }
+      if(end!=null) {
+        endTime = sdf.parse(end).getTime();
+      } else {
+        endTime = time.getEndTime();
+      }
+      if(rkey!=null) {
+        series = ChukwaHBaseStore.getSeries(table, rkey, column, startTime, endTime, true);
+        buffer = series.toString();
+      } else {
+        throw new WebApplicationException(Response.status(Response.Status.BAD_REQUEST)
+            .entity("No row key defined.").build());
+      }
+    } catch (ParseException e) {
+      throw new WebApplicationException(Response.status(Response.Status.BAD_REQUEST)
+          .entity("Start/End date parse error.  Format: yyyyMMddHHmmss.").build());
+    }
+    return buffer;
+  }
+
+  @GET
+  @Path("series/{table}/{column}/session/{sessionKey}")
+  @Produces("application/json")
+  public String getSeriesBySessionAttribute(@Context HttpServletRequest request, @PathParam("table") String table, @PathParam("column") String column, @PathParam("sessionKey") String skey, @QueryParam("start") String start, @QueryParam("end") String end) {
+    SimpleDateFormat sdf = new SimpleDateFormat("yyyyMMddHHmmss");
+    String buffer = "";
+    Series series;
+    long startTime = 0;
+    long endTime = 0;
+    TimeHandler time = new TimeHandler(request);
+    try {
+      if(start!=null) {
+        startTime = sdf.parse(start).getTime();
+      } else {
+        startTime = time.getStartTime();
+      }
+      if(end!=null) {
+        endTime = sdf.parse(end).getTime();
+      } else {
+        endTime = time.getEndTime();
+      }
+      if(skey!=null) {
+          HttpSession session = request.getSession();
+          String[] rkeys = (session.getAttribute(skey).toString()).split(",");
+          JSONArray seriesList = new JSONArray();
+          for(String rowKey : rkeys) {
+            Series output = ChukwaHBaseStore.getSeries(table, rowKey, column, startTime, endTime, true);
+            seriesList.put(output.toJSONObject());
+          }
+          buffer = seriesList.toString();
+      } else {
+        throw new WebApplicationException(Response.status(Response.Status.BAD_REQUEST)
+            .entity("No session attribute key defined.").build());
+      }
+    } catch (ParseException e) {
+      throw new WebApplicationException(Response.status(Response.Status.BAD_REQUEST)
+          .entity("Start/End date parse error.  Format: yyyyMMddHHmmss.").build());
+    }
+    return buffer;
+  }
+
+  @GET
+  @Path("schema")
+  @Produces("application/json")
+  public String getTables() {
+    Set<String> tableNames = ChukwaHBaseStore.getTableNames();
+    JSONArray tables = new JSONArray();
+    for(String table : tableNames) {
+      tables.put(table);
+    }
+    return tables.toString();
+  }
+  
+  @GET
+  @Path("schema/{table}")
+  @Produces("application/json")
+  public String getFamilies(@PathParam("table") String tableName) {
+    Set<String> familyNames = ChukwaHBaseStore.getFamilyNames(tableName);
+    JSONArray families = new JSONArray();
+    for(String family : familyNames) {
+      families.put(family);
+    }
+    return families.toString();
+  }
+  
+  @GET
+  @Path("schema/{table}/{family}")
+  @Produces("application/json")
+  public String getColumnNames(@Context HttpServletRequest request, @PathParam("table") String tableName, @PathParam("family") String family, @QueryParam("start") String start, @QueryParam("end") String end) {
+    SimpleDateFormat sdf = new SimpleDateFormat("yyyyMMddHHmmss");
+    String buffer = "";
+    Series series;
+    long startTime = 0;
+    long endTime = 0;
+    TimeHandler time = new TimeHandler(request);
+    try {
+      if(start!=null) {
+        startTime = sdf.parse(start).getTime();
+      } else {
+        startTime = time.getStartTime();
+      }
+      if(end!=null) {
+        endTime = sdf.parse(end).getTime();
+      } else {
+        endTime = time.getEndTime();
+      }
+    } catch(ParseException e) {
+      throw new WebApplicationException(Response.status(Response.Status.BAD_REQUEST)
+          .entity("Start/End date parse error.  Format: yyyyMMddHHmmss.").build());      
+    }
+    Set<String> columnNames = ChukwaHBaseStore.getColumnNames(tableName, family, startTime, endTime);
+    JSONArray columns = new JSONArray();
+    for(String column : columnNames) {
+      columns.put(column);
+    }
+    return columns.toString();
+  }
+  
+  @GET
+  @Path("rowkey/{table}/{column}")
+  @Produces("application/json")
+  public String getRowNames(@Context HttpServletRequest request, @PathParam("table") String tableName, @PathParam("family") String family, @PathParam("column") String column, @QueryParam("start") String start, @QueryParam("end") String end) {
+    SimpleDateFormat sdf = new SimpleDateFormat("yyyyMMddHHmmss");
+    String buffer = "";
+    Series series;
+    long startTime = 0;
+    long endTime = 0;
+    TimeHandler time = new TimeHandler(request);
+    try {
+      if(start!=null) {
+        startTime = sdf.parse(start).getTime();
+      } else {
+        startTime = time.getStartTime();
+      }
+      if(end!=null) {
+        endTime = sdf.parse(end).getTime();
+      } else {
+        endTime = time.getEndTime();
+      }
+    } catch(ParseException e) {
+      throw new WebApplicationException(Response.status(Response.Status.BAD_REQUEST)
+          .entity("Start/End date parse error.  Format: yyyyMMddHHmmss.").build());      
+    }
+    Set<String> columnNames = ChukwaHBaseStore.getRowNames(tableName, column, startTime, endTime);
+    JSONArray rows = new JSONArray();
+    for(String row : columnNames) {
+      rows.put(row);
+    }
+    return rows.toString();
+  }
+}

Added: incubator/chukwa/trunk/src/web/hicc/js/autoHeight.js
URL: http://svn.apache.org/viewvc/incubator/chukwa/trunk/src/web/hicc/js/autoHeight.js?rev=1023544&view=auto
==============================================================================
--- incubator/chukwa/trunk/src/web/hicc/js/autoHeight.js (added)
+++ incubator/chukwa/trunk/src/web/hicc/js/autoHeight.js Sun Oct 17 18:20:33 2010
@@ -0,0 +1,34 @@
+function doIframe(){
+	o = document.getElementsByTagName('iframe');
+	for(i=0;i<o.length;i++){
+		if (/\bautoHeight\b/.test(o[i].className)){
+			setHeight(o[i]);
+			addEvent(o[i],'load', doIframe);
+		}
+	}
+}
+
+function setHeight(e){
+	if(e.contentDocument){
+		e.height = e.contentDocument.body.offsetHeight + 35;
+	} else {
+		e.height = e.contentWindow.document.body.scrollHeight;
+	}
+}
+
+function addEvent(obj, evType, fn){
+	if(obj.addEventListener)
+	{
+	obj.addEventListener(evType, fn,false);
+	return true;
+	} else if (obj.attachEvent){
+	var r = obj.attachEvent("on"+evType, fn);
+	return r;
+	} else {
+	return false;
+	}
+}
+
+if (document.getElementById && document.createTextNode){
+ addEvent(window,'load', doIframe);	
+}

Modified: incubator/chukwa/trunk/src/web/hicc/jsp/chart.jsp
URL: http://svn.apache.org/viewvc/incubator/chukwa/trunk/src/web/hicc/jsp/chart.jsp?rev=1023544&r1=1023543&r2=1023544&view=diff
==============================================================================
--- incubator/chukwa/trunk/src/web/hicc/jsp/chart.jsp (original)
+++ incubator/chukwa/trunk/src/web/hicc/jsp/chart.jsp Sun Oct 17 18:20:33 2010
@@ -63,7 +63,10 @@
      */
     String[] seriesName = xf.getParameterValues("series_name");
     String[] data = xf.getParameterValues("data");
-
+    if(xf.getParameterValues("data").length==1 && xf.getParameterValues("data")[0].indexOf(",")!=-1) {
+      data = xf.getParameterValues("data")[0].split(",");
+    }
+    
     /*
      * Set series render format.
      */

Added: incubator/chukwa/trunk/src/web/hicc/jsp/graph_explorer.jsp
URL: http://svn.apache.org/viewvc/incubator/chukwa/trunk/src/web/hicc/jsp/graph_explorer.jsp?rev=1023544&view=auto
==============================================================================
--- incubator/chukwa/trunk/src/web/hicc/jsp/graph_explorer.jsp (added)
+++ incubator/chukwa/trunk/src/web/hicc/jsp/graph_explorer.jsp Sun Oct 17 18:20:33 2010
@@ -0,0 +1,153 @@
+<%
+/*
+ * 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.
+ */
+%>
+<%@ page import = "java.text.DecimalFormat,java.text.NumberFormat" %>
+<%@ page import = "org.apache.hadoop.chukwa.util.XssFilter" %>
+
+<%
+    XssFilter xf = new XssFilter(request);
+    NumberFormat nf = new DecimalFormat("###,###,###,##0.00");
+    response.setHeader("boxId", xf.getParameter("boxId"));
+    response.setContentType("text/html; chartset=UTF-8//IGNORE");
+    String boxId=xf.getParameter("boxId");
+    String cluster = (String) session.getAttribute("cluster");
+%>
+<html>
+  <head>
+  <script src="/hicc/js/jquery-1.3.2.min.js" type="text/javascript" charset="utf-8"></script>
+  <script src="/hicc/js/autoHeight.js" type="text/javascript" charset="utf-8"></script>
+  <script>
+    $.ajax({ url: "/hicc/v1/metrics/schema", dataType: "json", success: function(data){
+      for(var i in data) {
+        $('#table').append("<option>"+data[i]+"</option>");
+      }
+    }});
+
+    function getFamilies() {
+      var size = $('#family option').size();
+      $('#family').find('option').remove();
+      var table = $('#table').val();
+      $.ajax({ url: "/hicc/v1/metrics/schema/"+table, dataType: "json", success: function(data){
+        for(var i in data) {
+          $('#family').append("<option>"+data[i]+"</option>");
+        }
+      }});
+    }
+
+    function getColumns() {
+      var size = $('#column option').size();
+      $('#column').find('option').remove();
+      var table = $('#table').val();
+      var family = $('#family').val();
+      $('#family :selected').each(function(i, selected) {
+        var family = $(selected).val();
+        var url = "/hicc/v1/metrics/schema/"+table+"/"+family;
+        $.ajax({ url: url, dataType: "json", success: function(data){
+          for(var i in data) {
+            $('#column').append("<option>"+data[i]+"</option>");
+          }
+        }});
+      });
+    }
+
+    function getRows() {
+      var size = $('#row option').size();
+      $('#row').find('option').remove();
+      var table = $('#table').val();
+      var family = $('#family').val();
+      var column = $('#column').val();
+      $('#column :selected').each(function(i, selected) {
+        var column = $(selected).val();
+        var url = "/hicc/v1/metrics/rowkey/"+table+"/"+column;
+        $.ajax({ url: url, dataType: "json", success: function(data){
+          for(var i in data) {
+            $('#row').not(":contains('"+data[i]+"')").append("<option>"+data[i]+"</option>");
+          }
+        }});
+      });
+    }
+
+    function plot() {
+      var test = $('#row').val();
+      if(test == null) {
+        $('#row option:eq(0)').attr('selected',true);
+      }
+      var data = [];
+      $('#column :selected').each(function(i, selected) {
+        data[i] = $(selected).val();
+      });
+      var url = [];
+      for(var i in data) {
+        url[i] = "/hicc/v1/metrics/series/" + $('#table').val() + "/" + data[i] + "/rowkey/" + $('#row').val();
+      } 
+      var title = $('#title').val();
+      $('#graph').attr('src', "/hicc/jsp/chart.jsp?title="+title+"&data="+url.join("&data="));
+      $('#graph').load(function() {
+        doIframe();
+      });
+    }
+  </script>
+  </head>
+  <body>
+    <form>
+      <center>
+      <table>
+        <tr>
+          <td colspan="3">
+          Title <input type=text id="title">
+          </td>
+        </tr>
+        <tr>
+          <td>
+            Table<br>
+            <select id="table" size="10" onMouseUp="getFamilies()">
+            </select>
+          </td>
+          <td>
+            Column Family<br>
+            <select id="family" multiple size="10" onMouseUp="getColumns()">
+            <option>test</option>
+            </select>
+          </td>
+          <td>
+            Column<br>
+            <select id="column" multiple size="10" onMouseUp="getRows()">
+            </select>
+          </td>
+          <td>
+            Row<br>
+            <select id="row" size="10">
+            </select>
+          </td>
+        </tr>
+        <tr>
+          <td>
+            <input type=button name="action" value="Plot" onClick="plot()">
+          </td>
+          <td>
+          </td>
+          <td>
+          </td>
+        </tr>
+      </table>
+    </form>
+    <iframe id="graph" width="95%" class="autoHeight" frameBorder="0" scrolling="no"></iframe>
+    </center>
+  </body>
+</html>

Modified: incubator/chukwa/trunk/src/web/hicc/jsp/host_selector_dropdown.jsp
URL: http://svn.apache.org/viewvc/incubator/chukwa/trunk/src/web/hicc/jsp/host_selector_dropdown.jsp?rev=1023544&r1=1023543&r2=1023544&view=diff
==============================================================================
--- incubator/chukwa/trunk/src/web/hicc/jsp/host_selector_dropdown.jsp (original)
+++ incubator/chukwa/trunk/src/web/hicc/jsp/host_selector_dropdown.jsp Sun Oct 17 18:20:33 2010
@@ -29,9 +29,15 @@
 <%@ page import = "org.apache.hadoop.chukwa.hicc.TimeHandler" %>
 <%@ page import = "org.apache.hadoop.chukwa.database.DatabaseConfig" %>
 <%@ page import = "org.apache.hadoop.chukwa.util.XssFilter"  %>
-<% XssFilter xf = new XssFilter(request);
-   String boxId = xf.getParameter("boxId");
-   response.setHeader("boxId", xf.getParameter("boxId"));
+<%@ page import = "org.apache.hadoop.chukwa.datastore.ChukwaHBaseStore"  %>
+<%@ page import = "org.apache.commons.logging.Log" %>
+<%@ page import = "org.apache.commons.logging.LogFactory" %>
+<%@ page import = "org.apache.hadoop.chukwa.util.ExceptionUtil" %>
+<%
+  Log log = LogFactory.getLog(this.getClass());
+  XssFilter xf = new XssFilter(request);
+  String boxId = xf.getParameter("boxId");
+  response.setHeader("boxId", xf.getParameter("boxId"));
 %>
 <div class="panel">
 <h2>Hosts</h2>
@@ -41,123 +47,42 @@
 <%
     JSONArray machineNames = null;
     if(session.getAttribute("cache.machine_names")!=null) {
-        machineNames = new JSONArray(session.getAttribute("cache.machine_names").toString());
+      machineNames = new JSONArray(session.getAttribute("cache.machine_names").toString());
     }
     String cluster=xf.getParameter("cluster");
     if(cluster!=null && !cluster.equals("null")) {
-        session.setAttribute("cluster",cluster);
+      session.setAttribute("cluster",cluster);
     } else {
-        cluster = (String) session.getAttribute("cluster");
-        if(cluster==null || cluster.equals("null")) {
-            cluster="demo";
-            session.setAttribute("cluster",cluster);
-        }
+      cluster = (String) session.getAttribute("cluster");
+      if(cluster==null || cluster.equals("null")) {
+          cluster="demo";
+          session.setAttribute("cluster",cluster);
+      }
     }
     ClusterConfig cc = new ClusterConfig();
-    String jdbc = cc.getURL(cluster);
     TimeHandler time = new TimeHandler(request,(String)session.getAttribute("time_zone"));
     String startS = time.getStartTimeText();
     String endS = time.getEndTimeText();
-    String timefield = "timestamp";
-    String dateclause = timefield+" >= '"+startS+"' and "+timefield+" <= '"+endS+"'";
-    Connection conn = null;
-    Statement stmt = null;
-    ResultSet rs = null;
-    String query = "";
     try {
-        HashMap<String, String> hosts = new HashMap<String, String>();
-        try {
-            String[] selected_hosts = ((String)session.getAttribute("hosts")).split(",");
-            for(String name: selected_hosts) {
-                hosts.put(name,name);
-            }
-        } catch (NullPointerException e) {
+      HashMap<String, String> hosts = new HashMap<String, String>();      
+      try {
+        String[] selected_hosts = ((String)session.getAttribute("hosts")).split(",");
+        for(String name: selected_hosts) {
+          hosts.put(name,name);
+        }
+      } catch (NullPointerException e) {
+      }
+      Set<String> machines = ChukwaHBaseStore.getHostnames(cluster, time.getStartTime(), time.getEndTime());
+      for(String machine : machines) {
+        if(hosts.containsKey(machine)) {
+          out.println("<option selected>"+machine+"</option>");
+        } else {
+          out.println("<option>"+machine+"</option>");
+        }
+      }
+    } catch (Exception e) {
+      log.error(ExceptionUtil.getStackTrace(e));
     }
-           conn = org.apache.hadoop.chukwa.util.DriverManagerUtil.getConnection(jdbc);
-           stmt = conn.createStatement();
-           String jobId = (String)session.getAttribute("JobID");
-           if(jobId!=null && !jobId.equals("null") && !jobId.equals("")) {
-               query = "select DISTINCT Machine from HodMachine where HodID='"+jobId+"' order by Machine;";
-           } else if(machineNames==null) {
-               long start = time.getStartTime();
-               long end = time.getEndTime(); 
-               String table = "system_metrics";
-               DatabaseConfig dbc = new DatabaseConfig();
-               String[] tables = dbc.findTableNameForCharts(table, start, end);
-               table=tables[0];
-               query="select DISTINCT host from "+table+" order by host";
-           }
-           // or alternatively, if you don't know ahead of time that
-           // the query will be a SELECT...
-           if(!query.equals("")) {
-               if (stmt.execute(query)) {
-                   int i=0;
-                   rs = stmt.getResultSet();
-                   rs.last();
-                   int size = rs.getRow();
-                   machineNames = new JSONArray();
-                   rs.beforeFirst();
-                   while (rs.next()) {
-                       String machine = rs.getString(1);
-                       machineNames.put(machine);
-                       if(hosts.containsKey(machine)) {
-                           out.println("<option selected>"+machine+"</option>");
-                       } else {
-                           out.println("<option>"+machine+"</option>");
-                       }
-                       i++;
-                   }
-                   if(jobId==null || jobId.equals("null") || jobId.equals("")) {
-                       session.setAttribute("cache.machine_names",machineNames.toString());
-                   }
-               }
-           } else {
-                   for(int j=0;j<machineNames.length();j++) {
-                       String machine = machineNames.get(j).toString();
-                       if(hosts.containsKey(machine)) {
-                           out.println("<option selected>"+machine+"</option>");
-                       } else {
-                           out.println("<option>"+machine+"</option>");
-                       }
-                   }
-           }
-           // Now do something with the ResultSet ....
-       } catch (SQLException ex) {
-           // handle any errors
-           // FIXME: should we use Log4j here?
-           System.out.println("SQLException on query " + query +" " + ex.getMessage());
-           System.out.println("SQLState: " + ex.getSQLState());
-           System.out.println("VendorError: " + ex.getErrorCode());
-       } finally {
-           // it is a good idea to release
-           // resources in a finally{} block
-           // in reverse-order of their creation
-           // if they are no-longer needed
-           if (rs != null) {
-               try {
-                   rs.close();
-               } catch (SQLException sqlEx) {
-                   // ignore
-               }
-               rs = null;
-           }
-           if (stmt != null) {
-               try {
-                   stmt.close();
-               } catch (SQLException sqlEx) {
-                   // ignore
-               }
-               stmt = null;
-           }
-           if (conn != null) {
-               try {
-                   conn.close();
-               } catch (SQLException sqlEx) {
-                   // ignore
-               }
-               conn = null;
-           }
-       }
 %>
 </select></div>
 <div class="row">