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/03/31 22:48:38 UTC

svn commit: r760607 - in /hadoop/chukwa/trunk: ./ bin/ src/java/org/apache/hadoop/chukwa/extraction/database/ src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/ src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/ src/java/org/apac...

Author: eyang
Date: Tue Mar 31 20:48:38 2009
New Revision: 760607

URL: http://svn.apache.org/viewvc?rev=760607&view=rev
Log:
CHUKWA-59. Collect HDFS Usage information for users. (Contribute by Cheng Zhang via Eric Yang)

Added:
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/JPluginMapper.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/LogEntry.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/Employee.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsageMetrics.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsageMetricsList.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsagePlugin.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/OrgChart.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/ChukwaMetrics.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/ChukwaMetricsList.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/GenericChukwaMetricsList.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPlugin.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPluginAgent.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPluginStatusMetricsList.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/inputtools/hdfsusage/
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsagePluginTest.java
    hadoop/chukwa/trunk/tools/service/chukwa-hdfsusage/
    hadoop/chukwa/trunk/tools/service/chukwa-hdfsusage/run
Modified:
    hadoop/chukwa/trunk/bin/exec-data-loader.sh
    hadoop/chukwa/trunk/bin/processSinkFiles.sh
    hadoop/chukwa/trunk/bin/systemDataLoader.sh
    hadoop/chukwa/trunk/build.xml
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/database/MetricDataLoader.java

Modified: hadoop/chukwa/trunk/bin/exec-data-loader.sh
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/bin/exec-data-loader.sh?rev=760607&r1=760606&r2=760607&view=diff
==============================================================================
--- hadoop/chukwa/trunk/bin/exec-data-loader.sh (original)
+++ hadoop/chukwa/trunk/bin/exec-data-loader.sh Tue Mar 31 20:48:38 2009
@@ -47,6 +47,9 @@
   if [ "X$PARM" == "Xtorque" ]; then
     kill -9 `cat ${CHUKWA_PID_DIR}/TorqueDataLoader.pid`
   fi
+  if [ "X$PARM" == "XHDFSUsage" ]; then
+    kill -9 `cat ${CHUKWA_PID_DIR}/HDFSUsage-data-loader.pid`
+  fi
   echo "done"
   exit 0
 }
@@ -87,5 +90,9 @@
   ${JAVA_HOME}/bin/java -DDOMAIN=${DOMAIN} -DTORQUE_SERVER=${TORQUE_SERVER} -DTORQUE_HOME=${TORQUE_HOME} -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DCHUKWA_LOG_DIR=${CHUKWA_LOG_DIR} -DRECORD_TYPE=Torque -Dlog4j.configuration=system-data-loader.properties -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.inputtools.mdl.TorqueDataLoader &
 fi
 
+if [ "X$PARM" == "XHDFSUsage" ]; then
+  ${JAVA_HOME}/bin/java $JVM_OPTS -DPERIOD=600 -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DCHUKWA_LOG_DIR=${CHUKWA_LOG_DIR} -DRECORD_TYPE=HDFSUsage -Dlog4j.configuration=system-data-loader.properties -classpath ${HADOOP_CONF_DIR}:${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.jplugin.JPluginAgent org.apache.hadoop.chukwa.inputtools.hdfsusage.HDFSUsagePlugin &
+fi
+
 pid=$!
 wait $pid

Modified: hadoop/chukwa/trunk/bin/processSinkFiles.sh
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/bin/processSinkFiles.sh?rev=760607&r1=760606&r2=760607&view=diff
==============================================================================
--- hadoop/chukwa/trunk/bin/processSinkFiles.sh (original)
+++ hadoop/chukwa/trunk/bin/processSinkFiles.sh Tue Mar 31 20:48:38 2009
@@ -124,7 +124,7 @@
   debugDate=`date `
   echo "$debugDate done with demux job" >> "${CHUKWA_LOG_DIR}/mr.log"
    
-  ${JAVA_HOME}/bin/java -Djava.library.path=${JAVA_LIBRARY_PATH} -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DCHUKWA_LOG_DIR=${CHUKWA_LOG_DIR} -Dlog4j.configuration=log4j.properties -classpath ${CLASSPATH}:${CHUKWA_CORE}:${HADOOP_JAR}:${COMMON}:${tools}:${CHUKWA_HOME}/conf org.apache.hadoop.chukwa.extraction.database.DatabaseLoader "${srcDoneHdfsDir}/demux" SystemMetrics Df Hadoop_dfs Hadoop_jvm Hadoop_mapred Hadoop_rpc MSSRGraph MRJobCounters NodeActivity HodJob HodMachine Hadoop_dfs_FSDirectory Hadoop_dfs_FSNamesystem Hadoop_dfs_datanode Hadoop_dfs_namenode Hadoop_jvm_metrics Hadoop_mapred_job Hadoop_mapred_jobtracker Hadoop_mapred_shuffleOutput Hadoop_mapred_tasktracker Hadoop_rpc_metrics
+  ${JAVA_HOME}/bin/java -Djava.library.path=${JAVA_LIBRARY_PATH} -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DCHUKWA_LOG_DIR=${CHUKWA_LOG_DIR} -Dlog4j.configuration=log4j.properties -classpath ${CLASSPATH}:${CHUKWA_CORE}:${HADOOP_JAR}:${COMMON}:${tools}:${CHUKWA_HOME}/conf org.apache.hadoop.chukwa.extraction.database.DatabaseLoader "${srcDoneHdfsDir}/demux" SystemMetrics Df Hadoop_dfs Hadoop_jvm Hadoop_mapred Hadoop_rpc MSSRGraph MRJobCounters NodeActivity HodJob HodMachine Hadoop_dfs_FSDirectory Hadoop_dfs_FSNamesystem Hadoop_dfs_datanode Hadoop_dfs_namenode Hadoop_jvm_metrics Hadoop_mapred_job Hadoop_mapred_jobtracker Hadoop_mapred_shuffleOutput Hadoop_mapred_tasktracker Hadoop_rpc_metrics HDFSUsage
   endDbLoaderTime=`date +%s`
   dbLoaderDuration=$(( $endDbLoaderTime - $endDemuxTime))
   echo "dbLoaderDuration $dbLoaderDuration" >> "${CHUKWA_LOG_DIR}/mr.log"

Modified: hadoop/chukwa/trunk/bin/systemDataLoader.sh
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/bin/systemDataLoader.sh?rev=760607&r1=760606&r2=760607&view=diff
==============================================================================
--- hadoop/chukwa/trunk/bin/systemDataLoader.sh (original)
+++ hadoop/chukwa/trunk/bin/systemDataLoader.sh Tue Mar 31 20:48:38 2009
@@ -87,6 +87,16 @@
     fi
   fi
 
+  EXISTS=0
+  pidFile="${CHUKWA_PID_DIR}/HDFSUsage-data-loader.pid"
+  if [ -f $pidFile ]; then
+    pid=`head ${pidFile}`
+    ChildPIDRunningStatus=`${JPS} | grep ${pid} | grep HDFSUsage | grep -v grep | wc -l`
+    if [ $ChildPIDRunningStatus -ge 1 ]; then
+      EXISTS=1
+    fi
+  fi
+
   if [ ${EXISTS} -lt 1 ]; then
     echo "df data loader is stopped."
     RESULT=1
@@ -131,6 +141,9 @@
   if [ -f ${CHUKWA_PID_DIR}/Netstat-data-loader.pid ]; then
     kill -9 `cat ${CHUKWA_PID_DIR}/Netstat-data-loader.pid`
   fi
+  if [ -f ${CHUKWA_PID_DIR}/HDFSUsage-data-loader.pid ]; then
+    kill -9 `cat ${CHUKWA_PID_DIR}/HDFSUsage-data-loader.pid`
+  fi
   rm -f $CHUKWA_PID_DIR/chukwa-$CHUKWA_IDENT_STRING-systemDataLoader.sh.pid
   echo "done"
   exit 0
@@ -227,5 +240,19 @@
   ${JAVA_HOME}/bin/java $JVM_OPTS -DPERIOD=60 -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DCHUKWA_LOG_DIR=${CHUKWA_LOG_DIR} -DRECORD_TYPE=Netstat -Dlog4j.configuration=system-data-loader.properties -classpath ${CLASSPATH}:${CHUKWA_CORE}:${HADOOP_JAR}:${COMMON}:${TOOLS}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.inputtools.plugin.metrics.Exec ${CHUKWA_HOME}/bin/netstat.sh &
 fi
 
+EXISTS=0
+pidFile="${CHUKWA_PID_DIR}/HDFSUsage-data-loader.pid"
+if [ -f $pidFile ]; then
+  pid=`head ${pidFile}`
+  ChildPIDRunningStatus=`${JPS} | grep ${pid} | grep HDFSUsage | grep -v grep | wc -l`
+  if [ $ChildPIDRunningStatus -ge 1 ]; then
+    EXISTS=1
+  fi
+fi
+
+if [ ${EXISTS} -lt 1 ]; then
+  ${JAVA_HOME}/bin/java $JVM_OPTS -DPERIOD=600 -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DCHUKWA_LOG_DIR=${CHUKWA_LOG_DIR} -DRECORD_TYPE=HDFSUsage -Dlog4j.configuration=system-data-loader.properties -classpath ${HADOOP_CONF_DIR}:${CLASSPATH}:${CHUKWA_CORE}:${HADOOP_JAR}:${COMMON}:${TOOLS}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.inputtools.jplugin.JPluginAgent org.apache.hadoop.chukwa.inputtools.hdfsusage.HDFSUsagePlugin &
+fi
+
 echo "done"
 

Modified: hadoop/chukwa/trunk/build.xml
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/build.xml?rev=760607&r1=760606&r2=760607&view=diff
==============================================================================
--- hadoop/chukwa/trunk/build.xml (original)
+++ hadoop/chukwa/trunk/build.xml Tue Mar 31 20:48:38 2009
@@ -891,8 +891,11 @@
 #sed 's:CHUKWA_CONF_DIR=/usr/local/chukwa/conf:CHUKWA_CONF_DIR=%{_conf_dir}:' > %{buildroot}/service/chukwa-pbsnodes/run
 #cat %{buildroot}%{_prefix}/tools/service/chukwa-torque/run | sed 's:/usr/local/chukwa:%{_prefix}:' | \
 #sed 's:CHUKWA_CONF_DIR=/usr/local/chukwa/conf:CHUKWA_CONF_DIR=%{_conf_dir}:' > %{buildroot}/service/chukwa-torque/run
-
+cat %{buildroot}%{_prefix}/tools/service/chukwa-hdfsusage/run | \
+sed 's:/usr/local/chukwa:%{_prefix}:' > %{buildroot}%{_prefix}/tools/service/chukwa-hdfsusage/run.new
+mv %{buildroot}%{_prefix}/tools/service/chukwa-hdfsusage/run.new %{buildroot}%{_prefix}/tools/service/chukwa-hdfsusage/run
 chmod a+x %{buildroot}/etc/init.d/chukwa-*
+chmod -R a+x %{buildroot}%{_prefix}/tools/service/chukwa-*
 chmod -R a+x %{buildroot}/service/chukwa-*
 rm -rf %{buildroot}%{_prefix}/src
 rm -rf %{buildroot}%{_prefix}/build.xml

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/database/MetricDataLoader.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/database/MetricDataLoader.java?rev=760607&r1=760606&r2=760607&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/database/MetricDataLoader.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/database/MetricDataLoader.java Tue Mar 31 20:48:38 2009
@@ -262,7 +262,7 @@
           String sqlValues = "";
           boolean firstValue = true;
           while (fi.hasNext()) {
-            String fieldKey = (String) fi.next();
+            String fieldKey = fi.next();
             if (transformer.containsKey(fieldKey)) {
               if (!firstValue) {
                 sqlValues = sqlValues + ", ";

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/JPluginMapper.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/JPluginMapper.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/JPluginMapper.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/JPluginMapper.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,60 @@
+/*
+ * 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.extraction.demux.processor.mapper;
+
+import java.util.Map.Entry;
+
+import org.apache.hadoop.chukwa.extraction.engine.ChukwaRecord;
+import org.apache.hadoop.chukwa.extraction.engine.ChukwaRecordKey;
+import org.apache.hadoop.chukwa.inputtools.jplugin.ChukwaMetrics;
+import org.apache.hadoop.chukwa.inputtools.jplugin.GenericChukwaMetricsList;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+
+public class JPluginMapper extends AbstractProcessor {
+  @Override
+  protected void parse(String recordEntry,
+      OutputCollector<ChukwaRecordKey, ChukwaRecord> output,
+      Reporter reporter) throws Throwable {
+    LogEntry entry = new LogEntry(recordEntry);
+    String xml = entry.getBody();
+    GenericChukwaMetricsList metricsList = new GenericChukwaMetricsList();
+    metricsList.fromXml(xml);
+    String recType = metricsList.getRecordType();
+    long timestamp = metricsList.getTimestamp();
+    for (ChukwaMetrics metrics : metricsList.getMetricsList()) {
+      key = new ChukwaRecordKey();
+      ChukwaRecord record = new ChukwaRecord();
+      this.buildGenericRecord(record, null, -1l, recType);
+      record.setTime(timestamp);
+      key.setKey(getKey(timestamp, metrics.getKey()));
+      record.add("key", metrics.getKey());
+      for (Entry<String, String> attr : metrics.getAttributes().entrySet()) {
+        record.add(attr.getKey(), attr.getValue());
+      }
+      output.collect(key, record);
+    }
+  }
+
+  private String getKey(long ts, String metricsKey) {
+    long unit = 60 * 60 * 1000;
+    long rounded = (ts / unit) * unit;
+    return rounded + "/" + metricsKey + "/" + ts;
+  }
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/LogEntry.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/LogEntry.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/LogEntry.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/extraction/demux/processor/mapper/LogEntry.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,47 @@
+package org.apache.hadoop.chukwa.extraction.demux.processor.mapper;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+public class LogEntry {
+	private final static SimpleDateFormat sdf = new SimpleDateFormat(
+			"yyyy-MM-dd HH:mm");
+
+	private Date date;
+	private String logLevel;
+	private String className;
+	private String body;
+
+	public LogEntry(String recordEntry) throws ParseException {
+		String dStr = recordEntry.substring(0, 23);
+		date = sdf.parse(dStr);
+		int start = 24;
+		int idx = recordEntry.indexOf(' ', start);
+		logLevel = recordEntry.substring(start, idx);
+		start = idx + 1;
+		idx = recordEntry.indexOf(' ', start);
+		className = recordEntry.substring(start, idx - 1);
+		body = recordEntry.substring(idx + 1);
+	}
+
+	public Date getDate() {
+		return date;
+	}
+
+	public void setDate(Date date) {
+		this.date = date;
+	}
+
+	public String getLogLevel() {
+		return logLevel;
+	}
+
+	public String getClassName() {
+		return className;
+	}
+
+	public String getBody() {
+		return body;
+	}
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/Employee.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/Employee.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/Employee.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/Employee.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.chukwa.inputtools.hdfsusage;
+
+public class Employee {
+  private String id;
+  private Employee manager;
+  
+  public void setId(String id) {
+    this.id = id;
+  }
+  public String getId() {
+    return id;
+  }
+  public void setManager(Employee manager) {
+    this.manager = manager;
+  }
+  public Employee getManager() {
+    return manager;
+  }
+
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsageMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsageMetrics.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsageMetrics.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsageMetrics.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,69 @@
+/*
+ * 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.inputtools.hdfsusage;
+
+import java.util.Date;
+import java.util.HashMap;
+
+import org.apache.hadoop.chukwa.inputtools.jplugin.ChukwaMetrics;
+
+public class HDFSUsageMetrics implements ChukwaMetrics {
+  private String name = null;
+  
+  private Long size;
+  private long lastModified;
+  
+  @Override
+  public String getKey() {
+    return getName();
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setSize(Long size) {
+    this.size = size;
+  }
+
+  public Long getSize() {
+    return size;
+  }
+
+  @Override
+  public HashMap<String, String> getAttributes() {
+    HashMap<String, String> attr = new HashMap<String, String>();
+    attr.put("user", name);
+    attr.put("bytes", size.toString());
+    attr.put("timestamp", new Date().getTime() + "");
+    return attr;
+  }
+
+  public void setLastModified(long lastModified) {
+    this.lastModified = lastModified;
+  }
+
+  public long getLastModified() {
+    return lastModified;
+  }
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsageMetricsList.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsageMetricsList.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsageMetricsList.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsageMetricsList.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,27 @@
+/*
+ * 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.inputtools.hdfsusage;
+
+import org.apache.hadoop.chukwa.inputtools.jplugin.ChukwaMetricsList;
+
+public class HDFSUsageMetricsList extends ChukwaMetricsList<HDFSUsageMetrics> {
+  public HDFSUsageMetricsList() {
+    super("HDFSUsage");
+  }
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsagePlugin.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsagePlugin.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsagePlugin.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsagePlugin.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,109 @@
+/*
+ * 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.inputtools.hdfsusage;
+
+import java.net.URI;
+
+import org.apache.hadoop.chukwa.inputtools.jplugin.ChukwaMetricsList;
+import org.apache.hadoop.chukwa.inputtools.jplugin.JPlugin;
+import org.apache.hadoop.chukwa.inputtools.jplugin.JPluginStatusMetricsList;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class HDFSUsagePlugin implements JPlugin<HDFSUsageMetrics> {
+  private FileSystem hdfs;
+  private String path;
+  private OrgChart chart;
+  
+  @Override
+  public ChukwaMetricsList<HDFSUsageMetrics> getMetrics() throws Throwable {
+    ChukwaMetricsList<HDFSUsageMetrics> metricsList = new ChukwaMetricsList<HDFSUsageMetrics>(getRecordType());
+    FileStatus status[] = hdfs.globStatus(new Path(path));
+    for(int i=0; i<status.length; i++) {
+      long totalSize = hdfs.getContentSummary(status[i].getPath()).getLength();
+      if(totalSize <= 0) {
+        continue;
+      }
+      String name = status[i].getPath().getName();
+      HDFSUsageMetrics usage = new HDFSUsageMetrics();
+      usage.setName(name);
+      usage.setSize(totalSize);
+      usage.setLastModified(status[i].getModificationTime());
+      metricsList.addMetrics(usage);
+      
+      // also contribute to manager's usage
+      if(chart != null) {
+        Employee employee = chart.get(name);
+        if(employee != null) {
+          employee = employee.getManager();
+          while(employee != null) {
+            HDFSUsageMetrics managerUsage = new HDFSUsageMetrics();
+            managerUsage.setName(employee.getId());
+            managerUsage.setSize(totalSize);
+            metricsList.addMetrics(managerUsage);
+            employee = employee.getManager();
+          }
+        }
+      }
+    }
+    return metricsList;
+  }
+
+  @Override
+  public void init(String[] args) throws Throwable {
+    for(int i=0; i<args.length; i++) {
+      if(args[i].equals("-c")) {
+        String orgChartClass = args[i+1];
+        chart = (OrgChart) Class.forName(orgChartClass).newInstance();
+        i++;
+      } else if(args[i].equals("-h")) {
+        Configuration conf = new Configuration();
+        hdfs = FileSystem.get(new URI(args[i+1]), conf);
+        i++;
+      } else if(args[i].equals("-p")) {
+        path = args[i+1];
+        i++;
+      }
+    }
+    
+    if(hdfs == null) {
+      Configuration conf = new Configuration();
+      hdfs = FileSystem.get(conf);
+    }
+    
+    if(path == null) {
+      path = "/user/*";
+    }
+  }
+
+  @Override
+  public JPluginStatusMetricsList getStatus() throws Throwable {
+    JPluginStatusMetricsList list = new JPluginStatusMetricsList(this.getClass().getSimpleName());
+    list.addStatus("hdfs", hdfs.getUri().toString());
+    list.addStatus("path", path);
+    return null;
+  }
+
+  @Override
+  public String getRecordType() {
+    return "HDFSUsage";
+  }
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/OrgChart.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/OrgChart.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/OrgChart.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/hdfsusage/OrgChart.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,23 @@
+/*
+ * 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.inputtools.hdfsusage;
+
+public interface OrgChart {
+  Employee get(String id);
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/ChukwaMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/ChukwaMetrics.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/ChukwaMetrics.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/ChukwaMetrics.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,28 @@
+/*
+ * 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.inputtools.jplugin;
+
+import java.util.HashMap;
+
+import org.w3c.dom.Element;
+
+public interface ChukwaMetrics {
+  String getKey();
+  HashMap<String, String> getAttributes();
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/ChukwaMetricsList.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/ChukwaMetricsList.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/ChukwaMetricsList.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/ChukwaMetricsList.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,106 @@
+/*
+ * 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.inputtools.jplugin;
+
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map.Entry;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+
+public class ChukwaMetricsList<T extends ChukwaMetrics> {
+  private ArrayList<T> metricsList = null;
+  private long timestamp = new Date().getTime();
+  private String recordType = "JPlugin";
+
+  public ChukwaMetricsList() {
+  }
+  
+  public ChukwaMetricsList(String recordType) {
+    setRecordType(recordType);
+  }
+  
+  public void setMetricsList(ArrayList<T> metricsList) {
+    this.metricsList = metricsList;
+  }
+
+  public ArrayList<T> getMetricsList() {
+    if(metricsList == null){
+      metricsList = new ArrayList<T>();
+    }
+    return metricsList;
+  }
+
+  public void addMetrics(T metrics) {
+    getMetricsList().add(metrics);
+  }
+
+  public long getTimestamp() {
+    return timestamp;
+  }
+
+  public void setTimestamp(long timestamp) {
+    this.timestamp = timestamp;
+  }
+  
+  public String toXml() throws Exception {
+    DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+    DocumentBuilder docBuilder = null;
+    docBuilder = factory.newDocumentBuilder();
+    Document doc = docBuilder.newDocument();
+    Element root = doc.createElement(getRecordType());
+    doc.appendChild(root);
+    root.setAttribute("ts", getTimestamp()+"");
+    for(ChukwaMetrics metrics : getMetricsList()) {
+      Element elem = doc.createElement("Metrics");
+      elem.setAttribute("key", metrics.getKey());
+      for(Entry<String, String> attr : metrics.getAttributes().entrySet()) {
+        elem.setAttribute(attr.getKey(), attr.getValue());
+      }
+      root.appendChild(elem);
+    }
+    
+    Transformer transformer = TransformerFactory.newInstance().newTransformer();
+    transformer.setOutputProperty("indent", "yes");
+    StringWriter sw = new StringWriter();
+    transformer.transform(new DOMSource(doc), new StreamResult(sw));
+    
+    return sw.toString();
+  }
+
+  public void setRecordType(String recordType) {
+    this.recordType = recordType;
+  }
+
+  public String getRecordType() {
+    return recordType;
+  }
+  
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/GenericChukwaMetricsList.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/GenericChukwaMetricsList.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/GenericChukwaMetricsList.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/GenericChukwaMetricsList.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,105 @@
+/*
+ * 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.inputtools.jplugin;
+
+import java.io.StringReader;
+import java.util.HashMap;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+
+import org.w3c.dom.Attr;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NamedNodeMap;
+import org.w3c.dom.NodeList;
+import org.xml.sax.InputSource;
+
+public class GenericChukwaMetricsList extends ChukwaMetricsList<ChukwaMetrics> {
+  private static DocumentBuilderFactory factory;
+  private static DocumentBuilder docBuilder;
+  static {
+    factory = DocumentBuilderFactory.newInstance();
+    try {
+      docBuilder = factory.newDocumentBuilder();
+    } catch (ParserConfigurationException e) {
+      e.printStackTrace();
+    }
+  }
+  
+  public GenericChukwaMetricsList() {
+  }
+  
+  public GenericChukwaMetricsList(String recType) {
+    super(recType);
+  }
+  
+  public void fromXml(String xml) throws Exception {
+    InputSource is = new InputSource(new StringReader(xml));
+    Document doc = docBuilder.parse(is);
+    Element root = doc.getDocumentElement();
+    setRecordType(root.getTagName());
+    long timestamp = Long.parseLong(root.getAttribute("ts"));
+    setTimestamp(timestamp);
+    
+    NodeList children = root.getChildNodes();
+    for(int i=0; i<children.getLength(); i++) {
+      if(!children.item(i).getNodeName().equals("Metrics")) {
+        continue;
+      }
+      NamedNodeMap attrs = children.item(i).getAttributes();
+      if(attrs == null) {
+        continue;
+      }
+      
+      GenericChukwaMetrics metrics = new GenericChukwaMetrics();
+      for(int a=0; a<attrs.getLength(); a++) {
+        Attr attr = (Attr) attrs.item(a);
+        String name = attr.getName();
+        String value = attr.getValue();
+        if(name.equals("key")) {
+          metrics.setKey(value);
+        } else {
+          metrics.put(name, value);
+        }
+      }
+      addMetrics(metrics);
+    }
+  }
+
+  @SuppressWarnings("serial")
+  public static class GenericChukwaMetrics extends HashMap<String, String> implements ChukwaMetrics {
+    private String key;
+    
+    @Override
+    public HashMap<String, String> getAttributes() {
+      return this;
+    }
+
+    @Override
+    public String getKey() {
+      return key;
+    }
+
+    public void setKey(String key) {
+      this.key = key;
+    }
+  }
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPlugin.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPlugin.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPlugin.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPlugin.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,26 @@
+/*
+ * 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.inputtools.jplugin;
+
+public interface JPlugin<T extends ChukwaMetrics> {
+  String getRecordType();
+  ChukwaMetricsList<T> getMetrics() throws Throwable;
+  JPluginStatusMetricsList getStatus() throws Throwable;
+  void init(String[] args) throws Throwable;
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPluginAgent.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPluginAgent.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPluginAgent.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPluginAgent.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,129 @@
+/*
+ * 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.inputtools.jplugin;
+
+import java.util.Calendar;
+import java.util.Timer;
+import java.util.TimerTask;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.chukwa.util.PidFile;
+
+public class JPluginAgent {
+  private static Log log = LogFactory.getLog(JPluginAgent.class);
+
+  private static class MetricsTimerTask extends TimerTask {
+    @SuppressWarnings( { "unchecked" })
+    private JPlugin plugin;
+
+    @SuppressWarnings("unchecked")
+    public MetricsTimerTask(JPlugin plugin) {
+      this.plugin = plugin;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void run() {
+      try {
+        ChukwaMetricsList metrics = plugin.getMetrics();
+        String xml = metrics.toXml();
+        log.info(xml);
+      } catch (Throwable e) {
+        log.error(e.getMessage(), e);
+      }
+    }
+  }
+
+  private static class StatusTimerTask extends TimerTask {
+    @SuppressWarnings( { "unchecked" })
+    private JPlugin plugin;
+
+    @SuppressWarnings("unchecked")
+    public StatusTimerTask(JPlugin plugin) {
+      this.plugin = plugin;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void run() {
+      try {
+        ChukwaMetricsList metrics = plugin.getStatus();
+        String xml = metrics.toXml();
+        log.info(xml);
+      } catch (Throwable e) {
+        log.error(e.getMessage(), e);
+      }
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public static void main(String[] args) {
+    if (args.length < 1) {
+      System.out
+          .println("Usage: java -DPERIOD=nn JavaPluginAgent <class name> [parameters]");
+      System.exit(0);
+    }
+
+    int period = -1;
+    try {
+      if (System.getProperty("PERIOD") != null) {
+        period = Integer.parseInt(System.getProperty("PERIOD"));
+      }
+    } catch (NumberFormatException ex) {
+      ex.printStackTrace();
+      System.out.println("PERIOD should be numeric format of seconds.");
+      System.exit(0);
+    }
+
+    JPlugin plugin = null;
+    try {
+      plugin = (JPlugin) Class.forName(args[0]).newInstance();
+      plugin.init(args);
+    } catch (Throwable e) {
+      e.printStackTrace();
+      System.exit(-1);
+    }
+
+    try {
+      PidFile pFile = new PidFile(plugin.getRecordType() + "-data-loader");
+      Runtime.getRuntime().addShutdownHook(pFile);
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+
+    Calendar cal = Calendar.getInstance();
+    long now = cal.getTime().getTime();
+    cal.set(Calendar.SECOND, 3);
+    cal.set(Calendar.MILLISECOND, 0);
+    cal.add(Calendar.MINUTE, 1);
+    long until = cal.getTime().getTime();
+    try {
+      if (period == -1) {
+        new MetricsTimerTask(plugin).run();
+      } else {
+        Thread.sleep(until - now);
+        Timer timer = new Timer();
+        timer.scheduleAtFixedRate(new MetricsTimerTask(plugin), 0,
+            period * 1000);
+      }
+    } catch (Exception ex) {
+    }
+  }
+}

Added: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPluginStatusMetricsList.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPluginStatusMetricsList.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPluginStatusMetricsList.java (added)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/jplugin/JPluginStatusMetricsList.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,35 @@
+/*
+ * 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.inputtools.jplugin;
+
+public class JPluginStatusMetricsList extends GenericChukwaMetricsList {
+  private String pluginName;
+
+  public JPluginStatusMetricsList(String pluginName) {
+    super("JPluginStatus");
+    this.pluginName = pluginName;
+  }
+  
+  public void addStatus(String name, String value) {
+    GenericChukwaMetrics metrics = new GenericChukwaMetrics();
+    metrics.put(name, value);
+    metrics.setKey(pluginName);
+    addMetrics(metrics);
+  }
+}

Added: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsagePluginTest.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsagePluginTest.java?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsagePluginTest.java (added)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/inputtools/hdfsusage/HDFSUsagePluginTest.java Tue Mar 31 20:48:38 2009
@@ -0,0 +1,59 @@
+/*
+ * 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.inputtools.hdfsusage;
+
+import java.util.Map.Entry;
+
+import org.apache.hadoop.chukwa.inputtools.hdfsusage.HDFSUsageMetrics;
+import org.apache.hadoop.chukwa.inputtools.hdfsusage.HDFSUsagePlugin;
+import org.apache.hadoop.chukwa.inputtools.jplugin.ChukwaMetrics;
+import org.apache.hadoop.chukwa.inputtools.jplugin.ChukwaMetricsList;
+import org.apache.hadoop.chukwa.inputtools.jplugin.GenericChukwaMetricsList;
+
+import junit.framework.TestCase;
+
+public class HDFSUsagePluginTest extends TestCase {
+
+  public void testGetMetrics() throws Throwable {
+    HDFSUsagePlugin plugin = new HDFSUsagePlugin();
+    plugin.init(new String[0]);
+    ChukwaMetricsList<HDFSUsageMetrics> list = plugin.getMetrics();
+    System.out.println(list.getTimestamp());
+    for (ChukwaMetrics metrics : list.getMetricsList()) {
+      HDFSUsageMetrics usage = (HDFSUsageMetrics) metrics;
+      System.out.print(usage.getName());
+      System.out.println("size: " + usage.getSize());
+    }
+    System.out.println();
+
+    String xml = list.toXml();
+    System.out.println(xml);
+
+    GenericChukwaMetricsList gene = new GenericChukwaMetricsList(xml);
+    System.out.println(list.getTimestamp());
+    for (ChukwaMetrics metrics : gene.getMetricsList()) {
+      System.out.print(metrics.getKey());
+      for (Entry<String, String> entry : metrics.getAttributes().entrySet()) {
+        System.out.println(entry.getKey() + ": " + entry.getValue());
+      }
+    }
+    System.out.println();
+  }
+
+}

Added: hadoop/chukwa/trunk/tools/service/chukwa-hdfsusage/run
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/tools/service/chukwa-hdfsusage/run?rev=760607&view=auto
==============================================================================
--- hadoop/chukwa/trunk/tools/service/chukwa-hdfsusage/run (added)
+++ hadoop/chukwa/trunk/tools/service/chukwa-hdfsusage/run Tue Mar 31 20:48:38 2009
@@ -0,0 +1,5 @@
+#!/bin/sh
+CHUKWA_CONF_DIR=/usr/local/chukwa/conf
+
+exec setuidgid gmon /usr/local/chukwa/bin/exec-data-loader.sh --config $CHUKWA_CONF_DIR HDFSUsage
+