Ver código fonte

HADOOP-4709. Add several new features and bug fixes to Chukwa. Contributed by Jerome Boulon, Eric Yang, Andy Konwinski, and Ariel Rabkin.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@723855 13f79535-47bb-0310-9956-ffa450edef68
Christopher Douglas 16 anos atrás
pai
commit
449ee17148
100 arquivos alterados com 6314 adições e 2670 exclusões
  1. 10 0
      CHANGES.txt
  2. 1 0
      src/contrib/build.xml
  3. 24 0
      src/contrib/chukwa/bin/README
  4. 1 1
      src/contrib/chukwa/bin/VERSION
  5. 12 2
      src/contrib/chukwa/bin/agent.sh
  6. 37 3
      src/contrib/chukwa/bin/buildDailyArchive.sh
  7. 37 3
      src/contrib/chukwa/bin/buildHourlyArchive.sh
  8. 32 6
      src/contrib/chukwa/bin/chukwa-config.sh
  9. 15 9
      src/contrib/chukwa/bin/chukwa-daemon.sh
  10. 32 0
      src/contrib/chukwa/bin/dailyRolling.sh
  11. 63 0
      src/contrib/chukwa/bin/dbAdmin.sh
  12. 25 0
      src/contrib/chukwa/bin/dbLoader.sh
  13. 32 0
      src/contrib/chukwa/bin/dbSetup.sh
  14. 24 0
      src/contrib/chukwa/bin/dumpArchive.sh
  15. 24 0
      src/contrib/chukwa/bin/dumpDataType.sh
  16. 24 0
      src/contrib/chukwa/bin/dumpRecord.sh
  17. 28 0
      src/contrib/chukwa/bin/hourlyRolling.sh
  18. 11 5
      src/contrib/chukwa/bin/jettyCollector.sh
  19. 2 0
      src/contrib/chukwa/bin/netstat.sh
  20. 1 1
      src/contrib/chukwa/bin/nodeActivityDataLoader.sh
  21. 117 20
      src/contrib/chukwa/bin/processSinkFiles.sh
  22. 2 2
      src/contrib/chukwa/bin/slaves.sh
  23. 31 0
      src/contrib/chukwa/bin/start-agents.sh
  24. 6 3
      src/contrib/chukwa/bin/start-all.sh
  25. 2 4
      src/contrib/chukwa/bin/start-probes.sh
  26. 26 0
      src/contrib/chukwa/bin/stop-agents.sh
  27. 1 0
      src/contrib/chukwa/bin/stop-all.sh
  28. 32 4
      src/contrib/chukwa/bin/stop-data-processors.sh
  29. 0 1
      src/contrib/chukwa/bin/stop-probes.sh
  30. 44 9
      src/contrib/chukwa/bin/systemDataLoader.sh
  31. 2 2
      src/contrib/chukwa/bin/torqueDataLoader.sh
  32. 48 0
      src/contrib/chukwa/bin/validateDemux.sh
  33. 472 84
      src/contrib/chukwa/build.xml
  34. 0 24
      src/contrib/chukwa/chukwafy_jobhist.awk
  35. 18 0
      src/contrib/chukwa/conf/README
  36. 12 0
      src/contrib/chukwa/conf/aggregator.sql
  37. 8 7
      src/contrib/chukwa/conf/chukwa-agent-conf.xml.template
  38. 0 0
      src/contrib/chukwa/conf/chukwa-agents.template
  39. 107 0
      src/contrib/chukwa/conf/chukwa-demux-conf.xml
  40. 46 20
      src/contrib/chukwa/conf/chukwa-env.sh.template
  41. 31 0
      src/contrib/chukwa/conf/chukwa-hadoop-metrics-log4j.properties
  42. 19 0
      src/contrib/chukwa/conf/chukwa-log4j.properties
  43. 1 1
      src/contrib/chukwa/conf/collectors.template
  44. 602 0
      src/contrib/chukwa/conf/database_create_tables
  45. 25 26
      src/contrib/chukwa/conf/hadoop-log4j.properties
  46. 4 0
      src/contrib/chukwa/conf/hadoop-metrics.properties
  47. 1 0
      src/contrib/chukwa/conf/jdbc.conf
  48. 1 0
      src/contrib/chukwa/conf/jdbc.conf.template
  49. 2 13
      src/contrib/chukwa/conf/log4j.properties
  50. 0 1097
      src/contrib/chukwa/conf/mdl.xml
  51. 610 444
      src/contrib/chukwa/conf/mdl.xml.template
  52. 2 2
      src/contrib/chukwa/conf/system-data-loader.properties
  53. 10 0
      src/contrib/chukwa/docs/README
  54. 84 79
      src/contrib/chukwa/docs/paper/chukwa_08.tex
  55. 0 23
      src/contrib/chukwa/hadoop-packaging/build-tarball.sh
  56. 0 3
      src/contrib/chukwa/hadoop-packaging/patchhadoop.patch
  57. 201 0
      src/contrib/chukwa/lib/jstl-LICENSE.txt
  58. 344 0
      src/contrib/chukwa/lib/jstl-README.txt
  59. 2 2
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/Chunk.java
  60. 3 3
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/ChunkImpl.java
  61. 25 9
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/conf/ChukwaConfiguration.java
  62. 204 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/Aggregator.java
  63. 101 24
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/Consolidator.java
  64. 106 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/DataExpiration.java
  65. 244 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/DatabaseConfig.java
  66. 159 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/MetricsAggregation.java
  67. 126 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/TableCreator.java
  68. 4 4
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/ChunkQueue.java
  69. 22 7
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/DataFactory.java
  70. 7 1
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/Adaptor.java
  71. 45 12
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/ExecAdaptor.java
  72. 143 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/FileAdaptor.java
  73. 26 13
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/FileTailer.java
  74. 164 50
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/FileTailingAdaptor.java
  75. 38 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TerminatorThread.java
  76. 26 5
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/agent/AdaptorFactory.java
  77. 18 6
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/agent/AgentControlSocketListener.java
  78. 364 248
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/agent/ChukwaAgent.java
  79. 25 14
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/agent/MemLimitQueue.java
  80. 46 22
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/collector/CollectorStub.java
  81. 112 51
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/collector/servlet/ServletCollector.java
  82. 20 12
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/collector/servlet/ServletDiagnostics.java
  83. 2 1
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/connector/Connector.java
  84. 98 56
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/connector/http/HttpConnector.java
  85. 40 14
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/controller/ChukwaAgentController.java
  86. 50 17
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/sender/ChukwaHttpSender.java
  87. 23 6
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/sender/RetryListOfCollectors.java
  88. 6 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/test/ConsoleOutConnector.java
  89. 5 4
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/ChukwaWriter.java
  90. 106 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/ClientAck.java
  91. 13 3
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/ConsoleWriter.java
  92. 24 10
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/InMemoryWriter.java
  93. 248 178
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java
  94. 29 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/WriterException.java
  95. 132 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveBuilder.java
  96. 2 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveDailyOutputFormat.java
  97. 45 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveDataTypeOutputFormat.java
  98. 42 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveDataTypePartitioner.java
  99. 35 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveStreamNameOutputFormat.java
  100. 38 0
      src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveStreamNamePartitioner.java

+ 10 - 0
CHANGES.txt

@@ -63,6 +63,16 @@ Trunk (unreleased changes)
     HADOOP-4661. Add DistCh, a new tool for distributed ch{mod,own,grp}.
     (szetszwo)
 
+    HADOOP-4709. Add several new features and bug fixes to Chukwa.
+    (Jerome Boulon, Eric Yang, Andy Konwinski, Ariel Rabkin via cdouglas)
+      Added Hadoop Infrastructure Care Center (UI for visualize data collected by Chukwa)
+      Added FileAdaptor for streaming small file in one chunk
+      Added compression to archive and demux output
+      Added unit tests and validation for agent, collector, and demux map reduce job
+      Added database loader for loading demux output (sequence file) to jdbc connected database
+      Added algorithm to distribute collector load more evenly
+
+
   IMPROVEMENTS
 
     HADOOP-4234. Fix KFS "glue" layer to allow applications to interface

+ 1 - 0
src/contrib/build.xml

@@ -50,6 +50,7 @@
       <fileset dir="." includes="streaming/build.xml"/>
       <fileset dir="." includes="fairscheduler/build.xml"/>
       <fileset dir="." includes="capacity-scheduler/build.xml"/>
+      <fileset dir="." includes="chukwa/build.xml"/>
     </subant>
   </target>
   

+ 24 - 0
src/contrib/chukwa/bin/README

@@ -0,0 +1,24 @@
+Check for an updated copy of this README at http://wiki.apache.org/hadoop/Chukwa_Startup_and_Shutdown_Scripts
+
+start-all.sh - runs start-collectors.sh, start-agents.sh, start-probes.sh, start-data-processors.sh
+
+start-collectors.sh - start the chukwa collector daemon (jettyCollector.sh) on hosts listed in conf/collectors
+stop-collectors.sh - stop the chukwa collector daemon (jettyCollector.sh) on hosts listed in conf/collectors
+
+jettyCollector.sh - start the chukwa collector daemon on the current host
+
+start-agents.sh - start chukwa agent daemon (agent.sh) on all hosts listed in conf/chukwa-agents
+stop-agents.sh - stop chukwa agent daemon (agent.sh) on all hosts listed in conf/chukwa-agents
+
+agent.sh - start the chukwa agent on the current host
+
+start-probes.sh - runs, in this order, systemDataLoader.sh, torqueDataLoader.sh, nodeActivityDataLoader.sh
+
+systemDataLoader.sh - every 60 seconds run: sar, iostat, top, top, df, netstat. //TODO: figure out where these guys send their output
+torqueDataLoader.sh - //TODO: FILL THIS IN!
+nodeActivityDataLoader.sh - //TODO: FILL THIS IN!
+
+slaves.sh <command command_args ...> - run arbitrary commands on all hosts in conf/slaves
+
+jettycollector.sh - start a jetty based version of the Chukwa collector
+agent.sh - start the chukwa agent on the local machine

+ 1 - 1
src/contrib/chukwa/bin/VERSION

@@ -1 +1 @@
-0.1.0
+0.1.1

+ 12 - 2
src/contrib/chukwa/bin/agent.sh

@@ -21,6 +21,16 @@ bin=`cd "$bin"; pwd`
 
 . "$bin"/chukwa-config.sh
 
-echo "${pid}" > "$CHUKWA_HOME/var/run/Agent.pid"
+echo "hadoop jar for agent is " ${HADOOP_JAR}
+trap '${JPS} | grep ChukwaAgent | cut -f 1 -d" " | xargs kill -TERM ; exit 0' 1 2 15
 
-${JAVA_HOME}/bin/java -DCHUKWA_HOME=${CHUKWA_HOME} -classpath ${CLASSPATH}:${chukwaCore}:${chukwaAgent}:${HADOOP_JAR}:${CHUKWA_HOME}/conf:${HADOOP_HOME}/conf:${common} org.apache.hadoop.chukwa.datacollection.agent.ChukwaAgent $@
+if [ "X$1" = "Xstop" ]; then
+  echo -n "Shutting down agent..."
+  JETTY_PID=`${JPS} | grep ChukwaAgent | cut -f 1 -d" "`
+  kill -TERM ${JETTY_PID} >&/dev/null
+  echo "done"
+  exit 0
+fi
+
+
+${JAVA_HOME}/bin/java -Xms32M -Xmx64M -DAPP=agent -Dlog4j.configuration=chukwa-log4j.properties -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DCHUKWA_LOG_DIR=${CHUKWA_LOG_DIR} -classpath ${CLASSPATH}:${CHUKWA_AGENT}:${CHUKWA_CORE}:${HADOOP_JAR}:${COMMON}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.datacollection.agent.ChukwaAgent $@

+ 37 - 3
src/contrib/chukwa/bin/buildDailyArchive.sh

@@ -20,11 +20,45 @@ bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 . "$bin"/chukwa-config.sh
 
-echo "${pid}" > "$CHUKWA_HOME/var/run/BuildDailyArchive.pid"
+echo "${pid}" > "$CHUKWA_HOME/var/run/buildDailyArchive.pid"
 
 HADOOP_CONF_DIR="${HADOOP_HOME}/conf/"
 HADOOP_CMDE="${HADOOP_HOME}/bin/hadoop "
 
+while [ 1 ]
+ do
+  now=`date +%s`
+  strDate=`date +%m/%d/%y%n`
+  srcHourly="/chukwa/postprocess/srcDaily$now/"
 
-  $HADOOP_CMDE jar ${chukwaCore} org.apache.hadoop.chukwa.extraction.archive.ChuckwaArchiveBuilder Daily $srcEventHdfsDir ${chuwaRecordsRepository}
- 
+  echo "Running $strDate $now" >> "${CHUKWA_LOG_DIR}/daily.log"
+
+  echo "srcHourly: $srcHourly " >> "${CHUKWA_LOG_DIR}/daily.log"
+
+  $HADOOP_CMDE dfs -mkdir $srcHourly/hourly
+  echo "done with mkdir" >> "${CHUKWA_LOG_DIR}/daily.log"
+ 
+  $HADOOP_CMDE dfs -mv "/chukwa/archives/hourly/*.arc" ${srcHourly}/hourly/
+  echo "done with mv archives" >> "${CHUKWA_LOG_DIR}/daily.log"
+ 
+  # Build the archive
+  $HADOOP_CMDE jar ${CHUKWA_CORE} org.apache.hadoop.chukwa.extraction.archive.ChuckwaArchiveBuilder Daily $srcHourly/hourly $srcHourly/daily
+  echo "done with chuckwaArchiveBuilder" >> "${CHUKWA_LOG_DIR}/daily.log"
+  
+   ## Hourly Archive available call all processors
+   ##############  ############## 
+  
+   ##############  ############## 
+  
+  
+  ############## MERGE or MOVE ##############
+  
+  ############## MERGE or MOVE ##############
+  
+  
+  now=`date +%s`
+  strDate=`date +%m/%d/%y%n`
+  echo "Stopping ${strDate} ${now}" >> "${CHUKWA_LOG_DIR}/daily.log"
+
+  sleep 36000
+done

+ 37 - 3
src/contrib/chukwa/bin/buildHourlyArchive.sh

@@ -20,11 +20,45 @@ bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 . "$bin"/chukwa-config.sh
 
-echo "${pid}" > "$CHUKWA_HOME/var/run/BuildDailyArchive.pid"
+echo "${pid}" > "$CHUKWA_HOME/var/run/buildDailyArchive.pid"
 
 HADOOP_CONF_DIR="${HADOOP_HOME}/conf/"
 HADOOP_CMDE="${HADOOP_HOME}/bin/hadoop "
 
+while [ 1 ]
+ do
+  now=`date +%s`
+  strDate=`date +%m/%d/%y%n`
+  srcHourly="/chukwa/postprocess/srcHourly$now/"
 
-  $HADOOP_CMDE jar ${chukwaCore} org.apache.hadoop.chukwa.extraction.archive.ChuckwaArchiveBuilder Hourly $srcEventHdfsDir ${chuwaRecordsRepository}
- 
+  echo "Running $strDate $now" >> "${CHUKWA_LOG_DIR}/hourly.log"
+
+  echo "srcHourly: $srcHourly " >> "${CHUKWA_LOG_DIR}/hourly.log"
+
+  $HADOOP_CMDE dfs -mkdir $srcHourly/raw
+  echo "done with mkdir" >> "${CHUKWA_LOG_DIR}/hourly.log"
+ 
+  $HADOOP_CMDE dfs -mv "/chukwa/archives/raw/*.arc" ${srcHourly}/raw/
+  echo "done with mv archives" >> "${CHUKWA_LOG_DIR}/hourly.log"
+ 
+  # Build the archive
+  $HADOOP_CMDE jar ${CHUKWA_CORE} org.apache.hadoop.chukwa.extraction.archive.ChuckwaArchiveBuilder Hourly $srcHourly/arcFiles $srcHourly/hourly
+  echo "done with chuckwaArchiveBuilder" >> "${CHUKWA_LOG_DIR}/hourly.log"
+  
+   ## Hourly Archive available call all processors
+   ##############  ############## 
+  
+   ##############  ############## 
+  
+  
+  ############## MERGE or MOVE ##############
+  
+  ############## MERGE or MOVE ##############
+  
+  
+  now=`date +%s`
+  strDate=`date +%m/%d/%y%n`
+  echo "Stopping ${strDate} ${now}" >> "${CHUKWA_LOG_DIR}/hourly.log"
+
+  sleep 36000
+done

+ 32 - 6
src/contrib/chukwa/bin/chukwa-config.sh

@@ -75,7 +75,13 @@ then
     fi
 fi
 
-export CHUKWA_LOG_DIR="$CHUKWA_HOME/logs"
+if [ -z ${CHUKWA_LOG_DIR} ]; then
+    export CHUKWA_LOG_DIR="$CHUKWA_HOME/logs"
+fi
+
+if [ -z ${CHUKWA_PID_DIR} ]; then
+    export CHUKWA_PID_DIR="${CHUKWA_HOME}/var/run"
+fi
 
 CHUKWA_VERSION=`cat ${CHUKWA_HOME}/bin/VERSION`
 
@@ -90,15 +96,35 @@ if [ -f "${CHUKWA_CONF_DIR}/chukwa-env.sh" ]; then
 fi
 
 export DATACONFIG=${CHUKWA_CONF_DIR}/mdl.xml
-COMMON=`ls ${CHUKWA_HOME}/lib/*.jar`
+COMMON=`ls ${CHUKWA_HOME}/lib/*.jar ${CHUKWA_HOME}/hadoopjars/commons*.jar`
 export COMMON=`echo ${COMMON} | sed 'y/ /:/'`
-export CHUKWA_CORE=${CHUKWA_HOME}/chukwa-core-${CHUKWA_VERSION}.jar
-export CHUKWA_AGENT=${CHUKWA_HOME}/chukwa-agent-${CHUKWA_VERSION}.jar
-export HADOOP_JAR=`ls ${HADOOP_HOME}/hadoop-*-core.jar`
+export CHUKWA_CORE=${CHUKWA_HOME}/build/chukwa-core-${CHUKWA_VERSION}.jar
+export CHUKWA_AGENT=${CHUKWA_HOME}/build/chukwa-agent-${CHUKWA_VERSION}.jar
 export CURRENT_DATE=`date +%Y%m%d%H%M`
 
+if [ -z ${HADOOP_JAR} ]; then
+  if [ -z ${HADOOP_HOME} ]; then
+        export HADOOP_HOME=../../..
+    fi
+    if [ -d ${HADOOP_HOME} ]; then
+        export HADOOP_JAR=`ls ${HADOOP_HOME}/build/hadoop-*-core.jar`
+        if [ -z ${HADOOP_JAR} ]; then
+            echo "Please make sure hadoop-*-core.jar exists in ${HADOOP_HOME}/build"
+            exit -1
+        fi
+    else
+        if [ -d ${CHUKWA_HOME}/hadoopjars ]; then
+            echo "WARNING: neither HADOOP_HOME nor HADOOP_JAR is set we we are reverting to defaults in $CHUKWA_HOME/hadoopjars dir"
+            export HADOOP_JAR=`ls ${CHUKWA_HOME}/hadoopjars/hadoop-*-core.jar`
+        else
+            echo "Please make sure hadoop-*-core.jar exists in ${CHUKWA_HOME}/hadoopjars"
+            exit -1
+        fi
+    fi
+fi
+
 if [ -z "$JAVA_HOME" ] ; then
-    export JAVA_HOME=/usr/lib/j2sdk1.5-sun
+  echo ERROR! You forgot to set JAVA_HOME in conf/chukwa-env.sh   
 fi
 
 export JPS=${JAVA_HOME}/bin/jps

+ 15 - 9
src/contrib/chukwa/bin/chukwa-daemon.sh

@@ -98,25 +98,31 @@ case $startStop in
   (start)
     MAIL=`cat ${CHUKWA_HOME}/conf/alert.conf`
 
+    RANDOM=`date '+%s'`
+    PARTROL_HOUR=$[($RANDOM % 24)]
+    if [ ${PARTROL_HOUR} -gt 12 ]; then
+        PARTROL_HOUR2=$[${PARTROL_HOUR}-12]
+    else 
+        PARTROL_HOUR2=$[${PARTROL_HOUR}+12]
+    fi
     if [ "${WATCHDOG}" != "" ]; then
         mkdir -p ${CHUKWA_HOME}/var/tmp >&/dev/null
         crontab -l > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
         crontest=$?
 
         if [ "X${crontest}" != "X0" ]; then
-          cat > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE} << CRON
-MAILTO=${MAIL}
-1,30 * * * * ${CHUKWA_HOME}/bin/watchdog.sh
-CRON
+          echo "MAILTO=${MAIL}" > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
         else
-          grep -v "${CHUKWA_HOME}/bin/watchdog.sh" ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE} | grep -v MAILTO > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}.2
+          grep -v "${CHUKWA_HOME}/bin/watchdog.sh" ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE} | grep -v MAILTO | grep -v "cat ${CHUKWA_HOME}/var/run/watchdog.out" | grep -v ${CHUKWA_HOME}/tools/expire.sh > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}.2
           echo "MAILTO=${MAIL}" > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
           cat ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}.2 >> ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
           rm -f ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}.2
-          cat >> ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE} << CRON
-1,30 * * * * ${CHUKWA_HOME}/bin/watchdog.sh
-CRON
         fi
+        cat >> ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE} << CRON
+*/5 * * * * ${CHUKWA_HOME}/bin/watchdog.sh > ${CHUKWA_HOME}/var/run/watchdog.out
+1 ${PARTROL_HOUR},${PARTROL_HOUR2} * * * /bin/bash -c "cat ${CHUKWA_HOME}/var/run/watchdog.out; cat /dev/null > ${CHUKWA_HOME}/var/run/watchdog.out"
+15 3 * * * ${CHUKWA_HOME}/tools/expire.sh 10 ${CHUKWA_LOG_DIR} nowait
+CRON
 
         # save crontab
         echo -n "Registering watchdog.."
@@ -152,7 +158,7 @@ CRON
 
     if [ "${WATCHDOG}" != "" ]; then
         # remove watchdog
-        crontab -l | grep -v ${CHUKWA_HOME}/bin/watchdog.sh > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
+        crontab -l | grep -v ${CHUKWA_HOME}/bin/watchdog.sh | grep -v ${CHUKWA_HOME}/var/run/watchdog.out | grep -v ${CHUKWA_HOME}/tools/expire.sh > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
         crontab ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
         rm -f ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
     fi

+ 32 - 0
src/contrib/chukwa/bin/dailyRolling.sh

@@ -0,0 +1,32 @@
+#!/bin/sh
+# 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.
+
+pid=$$
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+. "$bin"/chukwa-config.sh
+
+HADOOP_CONF_DIR="${HADOOP_HOME}/conf/"
+HADOOP_CMDE="${HADOOP_HOME}/bin/hadoop "
+
+  $HADOOP_CMDE jar ${CHUKWA_CORE} org.apache.hadoop.chukwa.extraction.demux.DailyChukwaRecordRolling rollInSequence true deleteRawdata true
+
+  previousDay=`date --date="2 day ago" +%Y%m%d`
+  #previousDay=`date -v -2d +%Y%m%d`
+  echo "deleting /chukwa/postprocess/srcSink${previousDay}_*"
+  $HADOOP_CMDE dfs -rmr "/chukwa/postprocess/srcSink${previousDay}_*"
+

+ 63 - 0
src/contrib/chukwa/bin/dbAdmin.sh

@@ -0,0 +1,63 @@
+#!/bin/sh
+# 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.
+
+pid=$$
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/chukwa-config.sh
+
+if [ "$CHUKWA_IDENT_STRING" = "" ]; then
+  export CHUKWA_IDENT_STRING="$USER"
+fi
+
+trap 'rm -f $CHUKWA_HOME/var/run/chukwa-$CHUKWA_IDENT_STRING-dbAdmin.sh.pid ${CHUKWA_HOME}/var/run/dbAdmin.pid; exit 0' 1 2 15
+EXP_DATE=`date +%Y-%m-%d`
+
+JVM_OPTS="-DAPP=dbAdmin -Dlog4j.configuration=chukwa-log4j.properties -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DCHUKWA_LOG_DIR=${CHUKWA_LOG_DIR} -DDATACONFIG=${CHUKWA_CONF_DIR}/mdl.xml -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR}"
+
+echo "${pid}" > "${CHUKWA_HOME}/var/run/dbAdmin.pid"
+while [ 1 ]
+  do
+    start=`date +%s`
+    cat ${CHUKWA_CONF_DIR}/jdbc.conf | \
+    while read LINE; do
+        CLUSTER=`echo ${LINE} | cut -f 1 -d'='`
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.TableCreator ${EXP_DATE} 7 
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.TableCreator ${EXP_DATE} 30
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.TableCreator ${EXP_DATE} 91
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.TableCreator ${EXP_DATE} 365
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.TableCreator ${EXP_DATE} 3650
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.Aggregator 
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.Consolidator 
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.DataExpiration ${EXP_DATE} 7
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.DataExpiration ${EXP_DATE} 30
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.DataExpiration ${EXP_DATE} 91
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.DataExpiration ${EXP_DATE} 365
+        ${JAVA_HOME}/bin/java -DCLUSTER=${CLUSTER} ${JVM_OPTS} org.apache.hadoop.chukwa.database.DataExpiration ${EXP_DATE} 3650
+    done
+    end=`date +%s`
+    duration=$(( $end - $start ))
+    if [ $duration -lt 300 ]; then
+        sleep=$(( 300 - $duration ))
+        SLEEP_COUNTER=`expr $sleep / 5`
+        while [ $SLEEP_COUNTER -gt 1 ]; do
+            sleep 5
+            SLEEP_COUNTER=`expr $SLEEP_COUNTER - 1`
+        done
+    fi
+done

+ 25 - 0
src/contrib/chukwa/bin/dbLoader.sh

@@ -0,0 +1,25 @@
+#!/bin/sh
+# 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.
+
+pid=$$
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/chukwa-config.sh
+
+echo "${pid}" > "$CHUKWA_HOME/var/run/dbLoader.pid"
+${JAVA_HOME}/bin/java -DDATACONFIG=${CHUKWA_CONF_DIR}/mdl.xml -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.extraction.database.MetricDataLoader $1

+ 32 - 0
src/contrib/chukwa/bin/dbSetup.sh

@@ -0,0 +1,32 @@
+#!/bin/sh
+# 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.
+
+pid=$$
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/chukwa-config.sh
+
+EXP_DATE=`date +%Y-%m-%d`
+echo -n "SETUP Database partition..."
+echo "${pid}" > "$CHUKWA_HOME/var/run/dbSetup.pid"
+${JAVA_HOME}/bin/java -DCLUSTER=$1 -DDATACONFIG=${CHUKWA_CONF_DIR}/mdl.xml -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.database.TableCreator ${EXP_DATE} 7 #>/dev/null 2>&1
+${JAVA_HOME}/bin/java -DCLUSTER=$1 -DDATACONFIG=${CHUKWA_CONF_DIR}/mdl.xml -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.database.TableCreator ${EXP_DATE} 30 >/dev/null 2>&1
+${JAVA_HOME}/bin/java -DCLUSTER=$1 -DDATACONFIG=${CHUKWA_CONF_DIR}/mdl.xml -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.database.TableCreator ${EXP_DATE} 91 >/dev/null 2>&1
+${JAVA_HOME}/bin/java -DCLUSTER=$1 -DDATACONFIG=${CHUKWA_CONF_DIR}/mdl.xml -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.database.TableCreator ${EXP_DATE} 365 >/dev/null 2>&1
+echo "done"
+rm -f "$CHUKWA_HOME/var/run/dbSetup.pid"

+ 24 - 0
src/contrib/chukwa/bin/dumpArchive.sh

@@ -0,0 +1,24 @@
+#!/bin/sh
+# 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.
+
+pid=$$
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/chukwa-config.sh
+
+${JAVA_HOME}/bin/java -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.util.DumpArchive $1

+ 24 - 0
src/contrib/chukwa/bin/dumpDataType.sh

@@ -0,0 +1,24 @@
+#!/bin/sh
+# 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.
+
+pid=$$
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/chukwa-config.sh
+
+${JAVA_HOME}/bin/java -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.util.DumpDataType $@

+ 24 - 0
src/contrib/chukwa/bin/dumpRecord.sh

@@ -0,0 +1,24 @@
+#!/bin/sh
+# 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.
+
+pid=$$
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/chukwa-config.sh
+
+${JAVA_HOME}/bin/java -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.util.DumpRecord $1

+ 28 - 0
src/contrib/chukwa/bin/hourlyRolling.sh

@@ -0,0 +1,28 @@
+#!/bin/sh
+# 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.
+
+pid=$$
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+. "$bin"/chukwa-config.sh
+
+HADOOP_CONF_DIR="${HADOOP_HOME}/conf/"
+HADOOP_CMDE="${HADOOP_HOME}/bin/hadoop "
+
+  $HADOOP_CMDE jar ${CHUKWA_CORE} org.apache.hadoop.chukwa.extraction.demux.HourlyChukwaRecordRolling rollInSequence true deleteRawdata true
+
+

+ 11 - 5
src/contrib/chukwa/bin/jettyCollector.sh

@@ -21,11 +21,17 @@ bin=`cd "$bin"; pwd`
 
 . "$bin"/chukwa-config.sh
 
-echo "${pid}" > "${CHUKWA_PID_DIR}/Collector.pid"
+trap 'stop; exit 0' 1 2 15
 
+function stop {
+  echo -n "Shutting down Collector..."
+  ${JPS} | grep CollectorStub | cut -f 1 -d" " | xargs kill -TERM >&/dev/null
+  echo "done"
+  exit 0
+}
 
-if [ $# -lt 1 ]; then 
-${JAVA_HOME}/bin/java -classpath ${CLASSPATH}:${chukwaCore}:${common}:${HADOOP_JAR}:${CHUKWA_HOME}/conf:${HADOOP_HOME}/conf org.apache.hadoop.chukwa.datacollection.collector.CollectorStub 9310
-else
-${JAVA_HOME}/bin/java -classpath ${CLASSPATH}:${chukwaCore}:${common}:${HADOOP_JAR}:${CHUKWA_HOME}/conf:${HADOOP_HOME}/conf org.apache.hadoop.chukwa.datacollection.collector.CollectorStub $@
+if [ "X$1" = "Xstop" ]; then
+  stop
 fi
+
+${JAVA_HOME}/bin/java -DAPP=collector -Dlog4j.configuration=chukwa-log4j.properties -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DCHUKWA_LOG_DIR=${CHUKWA_LOG_DIR} -classpath ${CLASSPATH}:${CHUKWA_CORE}:${COMMON}:${HADOOP_JAR}:${CHUKWA_CONF_DIR} org.apache.hadoop.chukwa.datacollection.collector.CollectorStub 

+ 2 - 0
src/contrib/chukwa/bin/netstat.sh

@@ -0,0 +1,2 @@
+#!/bin/bash
+netstat -a | grep ESTABLISH | grep -v '        0      0'

+ 1 - 1
src/contrib/chukwa/bin/nodeActivityDataLoader.sh

@@ -40,7 +40,7 @@ if [ -f $pidFile ]; then
 fi
 
 if [ ${EXISTS} -lt 1 ]; then
-    ${JAVA_HOME}/bin/java -DPERIOD=600 -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DRECORD_TYPE=PbsNodes -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 "${nodeActivityCmde}" &
+    ${JAVA_HOME}/bin/java -DPERIOD=600 -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DCHUKWA_LOG_DIR=${CHUKWA_LOG_DIR} -DRECORD_TYPE=PbsNodes -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 "${nodeActivityCmde}" &
 fi
 
 

+ 117 - 20
src/contrib/chukwa/bin/processSinkFiles.sh

@@ -20,40 +20,137 @@ bin=`dirname "$0"`
 bin=`cd "$bin"; pwd`
 . "$bin"/chukwa-config.sh
 
-echo "${pid}" > "${CHUKWA_PID_DIR}/ProcessSinkFiles.pid"
+if [ "$CHUKWA_IDENT_STRING" = "" ]; then
+  export CHUKWA_IDENT_STRING="$USER"
+fi
+
+trap 'remove_cron;rm -f $CHUKWA_HOME/var/run/chukwa-$CHUKWA_IDENT_STRING-processSinkFiles.sh.pid ${CHUKWA_HOME}/var/run/ProcessSinkFiles.pid; exit 0' 1 2 15
+echo "${pid}" > "$CHUKWA_HOME/var/run/ProcessSinkFiles.pid"
 
 HADOOP_CONF_DIR="${HADOOP_HOME}/conf/"
 HADOOP_CMDE="${HADOOP_HOME}/bin/hadoop "
 
+function remove_cron {
+    mkdir -p ${CHUKWA_HOME}/var/tmp >&/dev/null
+    crontab -l | grep -v ${CHUKWA_HOME}/bin/hourlyRolling.sh > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
+    cat /tmp/cron.${CURRENT_DATE} | grep -v ${CHUKWA_HOME}/bin/dailyRolling.sh > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}.2
+    crontab ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}.2
+    rm -f ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
+    rm -f ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}.2
+}
+
+function add_cron {
+    mkdir -p ${CHUKWA_HOME}/var/tmp >&/dev/null
+    crontab -l > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
+    crontest=$?
+
+    if [ "X${crontest}" != "X0" ]; then
+      cat > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE} << CRON
+16 * * * * ${CHUKWA_HOME}/bin/hourlyRolling.sh >& ${CHUKWA_HOME}/logs/hourly.log
+30 1 * * * ${CHUKWA_HOME}/bin/dailyRolling.sh >& ${CHUKWA_HOME}/logs/dailyRolling.log
+CRON
+    else
+      grep -v "${CHUKWA_HOME}/bin/hourlyRolling.sh" ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}  | grep -v "${CHUKWA_HOME}/bin/dailyRolling.sh" > ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}.2
+      mv ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}.2 ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
+      cat >> ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE} << CRON
+16 * * * * ${CHUKWA_HOME}/bin/hourlyRolling.sh >& ${CHUKWA_HOME}/logs/hourly.log
+30 1 * * * ${CHUKWA_HOME}/bin/dailyRolling.sh >& ${CHUKWA_HOME}/logs/dailyRolling.log
+CRON
+    fi
+
+    # save crontab
+    echo -n "Registering cron jobs.."
+    crontab ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE} > /dev/null 2>&1
+    rm -f ${CHUKWA_HOME}/var/tmp/cron.${CURRENT_DATE}
+    echo "done"
+}
+
+if [ "X$1" = "Xstop" ]; then
+  echo -n "Shutting down processSinkFiles.sh..."
+  kill -TERM `cat ${CHUKWA_HOME}/var/run/ProcessSinkFiles.pid`
+  echo "done"
+  exit 0
+fi
+
+if [ "X$1" = "Xwatchdog" ]; then
+  add_cron
+fi
+
 while [ 1 ]
-do 
+ do
+  debugDate=`date `
+  startTime=`date +%s`
+  now=`date +%Y%m%d_%H_%M%S`
+  strDate=`date +%Y%m%d_%H_%M%S`
+  srcDoneHdfsDir="/chukwa/postprocess/srcSink$now/"
   
   
-  now=`date +%s`
-  strDate=`date +%m/%d/%y%n`
-  srcDoneHdfsDir="/chukwa/tmp/srcDone$now/"
-  srcEventHdfsDir="/chukwa/tmp/srcEvent$now/"
+  destArchiveDir=`date +%Y%m%d/%H/%M%S`
 
   echo "Running $strDate $now" >> "${CHUKWA_LOG_DIR}/mr.log"
 
-  echo "srcDoneHdfsDir: $srcDoneHdfsDir srcEventHdfsDir: $srcEventHdfsDir" >> "${MR_LOG}"
+  echo "srcDoneHdfsDir: $srcDoneHdfsDir " >> "${CHUKWA_LOG_DIR}/mr.log"
 
-  $HADOOP_CMDE dfs -mkdir $srcDoneHdfsDir
+  $HADOOP_CMDE dfs -mkdir $srcDoneHdfsDir/doneFile
   echo "done with mkdir" >> "${CHUKWA_LOG_DIR}/mr.log"
  
-  $HADOOP_CMDE dfs -mv "/chukwa/logs/*/*.done" $srcDoneHdfsDir
-  echo "done with mv logs" >> "${CHUKWA_LOG_DIR}/mr.log"
+  $HADOOP_CMDE dfs -mv "/chukwa/logs/*/*.done" ${srcDoneHdfsDir}/doneFile
+  endMoveTime=`date +%s`
+  moveDuration=$(( $endMoveTime - $startTime))
+  echo "moveDuration $moveDuration" >> "${CHUKWA_LOG_DIR}/mr.log"
+  debugDate=`date `
+  echo "$debugDate done with mv logs" >> "${CHUKWA_LOG_DIR}/mr.log"
  
-  $HADOOP_CMDE jar ${chukwaCore} org.apache.hadoop.chukwa.extraction.demux.Demux -r 2 $srcDoneHdfsDir $srcEventHdfsDir
-  echo "done with demux job" >> "${CHUKWA_LOG_DIR}/mr.log"
- 
-  $HADOOP_CMDE jar ${chukwaCore} org.apache.hadoop.chukwa.extraction.demux.MoveOrMergeLogFile $srcEventHdfsDir ${chuwaRecordsRepository}
-  echo "done with MoveOrMergeLogFile" >> "${CHUKWA_LOG_DIR}/mr.log"
-
+  # Build the archive
+  $HADOOP_CMDE jar  ${CHUKWA_CORE} org.apache.hadoop.chukwa.extraction.archive.ChukwaArchiveBuilder Stream ${srcDoneHdfsDir}/doneFile /chukwa/archives/raw/${destArchiveDir}
+  endArchiveTime=`date +%s`
+  archiveDuration=$(( $endArchiveTime - $endMoveTime))
+  echo "archiveDuration $archiveDuration" >> "${CHUKWA_LOG_DIR}/mr.log"
+  debugDate=`date `
+  echo "$debugDate done with chuckwaArchiveBuilder" >> "${CHUKWA_LOG_DIR}/mr.log"
+  
+  
+  ## Archive available call all processors
+  
+  
+  $HADOOP_CMDE jar  ${CHUKWA_CORE} org.apache.hadoop.chukwa.extraction.demux.Demux -Dmapred.compress.map.output=true -Dmapred.map.output.compression.codec=org.apache.hadoop.io.compress.LzoCodec -Dmapred.output.compress=true -Dmapred.output.compression.type=BLOCK -r 4 /chukwa/archives/raw/${destArchiveDir} ${srcDoneHdfsDir}/demux
+  endDemuxTime=`date +%s`
+  demuxDuration=$(( $endDemuxTime - $endArchiveTime))
+  echo "demuxDuration $demuxDuration" >> "${CHUKWA_LOG_DIR}/mr.log"
+  debugDate=`date `
+  echo "$debugDate done with demux job" >> "${CHUKWA_LOG_DIR}/mr.log"
+   
+  ${JAVA_HOME}/bin/java -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
+  endDbLoaderTime=`date +%s`
+  dbLoaderDuration=$(( $endDbLoaderTime - $endDemuxTime))
+  echo "dbLoaderDuration $dbLoaderDuration" >> "${CHUKWA_LOG_DIR}/mr.log"
+  debugDate=`date `
+  echo "$debugDate done with dbLoader job" >> "${CHUKWA_LOG_DIR}/mr.log"
+   
+  $HADOOP_CMDE jar ${CHUKWA_CORE} org.apache.hadoop.chukwa.extraction.demux.MoveToRepository ${srcDoneHdfsDir}/demux ${chuwaRecordsRepository}
+  endMoveToRepoTime=`date +%s`
+  moveToRepoDuration=$(( $endMoveToRepoTime - $endDbLoaderTime))
+  echo "moveToRepoDuration $moveToRepoDuration" >> "${CHUKWA_LOG_DIR}/mr.log"
+  debugDate=`date `
+  echo "$debugDate done with MoveToRepository" >> "${CHUKWA_LOG_DIR}/mr.log"
+  
   now=`date +%s`
   strDate=`date +%m/%d/%y%n`
-  echo "Stopping ${strDate} ${now}" >> "${CHUKWA_LOG_DIR}/mr.log"
-
- sleep 300
-
+  debugDate=`date `
+  echo "$debugDate Stopping ${strDate} ${now}" >> "${CHUKWA_LOG_DIR}/mr.log"
+  
+  endTime=`date +%s`
+  duration=$(( $endTime - $startTime))
+  echo "Duration: $duration s" >> "${CHUKWA_LOG_DIR}/mr.log"
+  
+  if [ $duration -lt 300 ]; then
+   sleepTime=$(( 300 - $duration)) 
+   echo "Sleep: $sleepTime s" >> "${CHUKWA_LOG_DIR}/mr.log"
+   SLEEP_COUNTER=`expr $sleepTime / 5`
+   while [ $SLEEP_COUNTER -gt 1 ]; do
+       sleep 5
+       SLEEP_COUNTER=`expr $SLEEP_COUNTER - 1`
+   done
+  fi
 done
+

+ 2 - 2
src/contrib/chukwa/bin/slaves.sh

@@ -21,7 +21,7 @@
 # Environment Variables
 #
 #   CHUKWA_SLAVES    File naming remote hosts.
-#     Default is ${CHUKWA_CONF_DIR}/chukwa-slaves.
+#     Default is ${CHUKWA_CONF_DIR}/chukwa-agents.
 #   CHUKWA_CONF_DIR  Alternate conf dir. Default is ${CHUKWA_HOME}/conf.
 #   CHUKWA_SLAVE_SLEEP Seconds to sleep between spawning remote commands.
 #   CHUKWA_SSH_OPTS Options passed to ssh when running remote commands.
@@ -51,7 +51,7 @@ fi
 
 if [ "$HOSTLIST" = "" ]; then
   if [ "$CHUKWA_SLAVES" = "" ]; then
-    export HOSTLIST="${CHUKWA_CONF_DIR}/chukwa-slaves"
+    export HOSTLIST="${CHUKWA_CONF_DIR}/chukwa-agents"
   else
     export HOSTLIST="${CHUKWA_SLAVES}"
   fi

+ 31 - 0
src/contrib/chukwa/bin/start-agents.sh

@@ -0,0 +1,31 @@
+#!/usr/bin/env bash
+
+# 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.
+
+
+# This script is modeled after the parallel script in Hadoop
+# Start chukwa agent daemons on cluster slaves
+# Run this on a collector node.
+
+usage="Usage: start-agent.sh"
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/chukwa-config.sh
+
+# start chukwa agents
+"$bin"/chukwa-daemons.sh --config $CHUKWA_CONF_DIR --hosts slaves start agent.sh

+ 6 - 3
src/contrib/chukwa/bin/start-all.sh

@@ -23,11 +23,14 @@ bin=`cd "$bin"; pwd`
 
 . "$bin"/chukwa-config.sh
 
+# start collectors
+"$bin"/start-collectors.sh --config $CHUKWA_CONF_DIR
+
+# start agents
+"$bin"/start-agents.sh --config $CHUKWA_CONF_DIR
+
 # start probes
 "$bin"/start-probes.sh --config $CHUKWA_CONF_DIR
 
 # start data processors
 "$bin"/start-data-processors.sh --config $CHUKWA_CONF_DIR
-
-# start collectors
-"$bin"/start-collectors.sh --config $CHUKWA_CONF_DIR

+ 2 - 4
src/contrib/chukwa/bin/start-probes.sh

@@ -27,15 +27,13 @@ bin=`cd "$bin"; pwd`
 
 . "$bin"/chukwa-config.sh
 
-# start agents
-"$bin"/chukwa-daemons.sh --config $CHUKWA_CONF_DIR --watchdog start agent.sh
 # start system data loader daemons
 "$bin"/chukwa-daemons.sh --config $CHUKWA_CONF_DIR start systemDataLoader.sh
 
 # start torque data loader daemons
-if [ ${TORQUE_HOME} != "" ]; then
+if [ "x${TORQUE_HOME}" != "x" ]; then
   "$bin"/chukwa-daemon.sh --config $CHUKWA_CONF_DIR start torqueDataLoader.sh
 fi
-if [ ${nodeActivityCmde} != "" ]; then
+if [ "x${nodeActivityCmde}" != "x" ]; then
   "$bin"/chukwa-daemon.sh --config $CHUKWA_CONF_DIR start nodeActivityDataLoader.sh
 fi

+ 26 - 0
src/contrib/chukwa/bin/stop-agents.sh

@@ -0,0 +1,26 @@
+#!/usr/bin/env bash
+
+# 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.
+
+
+# Stop collectors.  Run this on master node.
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/chukwa-config.sh
+
+"$bin"/chukwa-daemons.sh --config $CHUKWA_CONF_DIR --hosts slaves --watchdog stop agent.sh

+ 1 - 0
src/contrib/chukwa/bin/stop-all.sh

@@ -25,4 +25,5 @@ bin=`cd "$bin"; pwd`
 
 "$bin"/stop-probes.sh --config $CHUKWA_CONF_DIR
 "$bin"/stop-data-processors.sh --config $CHUKWA_CONF_DIR
+"$bin"/stop-agents.sh --config $CHUKWA_CONF_DIR
 "$bin"/stop-collectors.sh --config $CHUKWA_CONF_DIR

+ 32 - 4
src/contrib/chukwa/bin/stop-data-processors.sh

@@ -22,24 +22,52 @@ java=$JAVA_HOME/bin/java
 . "$bin"/chukwa-config.sh
 
 # stop processSinkFiles.sh
-pidFile=$CHUKWA_HOME/var/run/ProcessSinkFiles.pid
+pidFile=$CHUKWA_PID_DIR/ProcessSinkFiles.pid
 if [ -f $pidFile ]; then  
    echo -n "Shutting down Data Processors.."
    DP_PID=`head ${pidFile}`
    kill -TERM ${DP_PID}
-   rm ${pidFile}
+   for i in 1 2 5; do
+       test_pid=`ps ax | grep ${DP_PID} | grep -v grep | grep processSinkFiles.sh | wc -l`
+       if [ $test_pid -ge 1 ]; then
+           sleep $i
+           kill -TERM ${DP_PID}
+       else
+           break
+       fi
+   done
+   test_pid=`ps ax | grep ${DP_PID} | grep -v grep | grep processSinkFiles.sh | wc -l`
+   if [ $test_pid -ge 1 ]; then
+       kill -9 ${DBADMIN_PID} &>/dev/null
+   fi
+   rm -f ${pidFile}
+   rm -f $CHUKWA_PID_DIR/chukwa-$CHUKWA_IDENT_STRING-processSinkFiles.sh.pid
    echo "done"
 else
   echo " no $pidFile"
 fi
 
 # stop dbAdmin.sh
-pidFile=$CHUKWA_HOME/var/run/dbAdmin.pid
+pidFile=$CHUKWA_PID_DIR/dbAdmin.pid
 if [ -f $pidFile ]; then  
    echo -n "Shutting down Database Admin.."
    DBADMIN_PID=`head ${pidFile}`
    kill -TERM ${DBADMIN_PID}
-   rm ${pidFile}
+   for i in 1 2 5; do
+       test_pid=`ps ax | grep ${DBADMIN_PID} | grep -v grep | grep dbAdmin.sh | wc -l`
+       if [ $test_pid -ge 1 ]; then
+           sleep $i
+           kill -TERM ${DBADMIN_PID}
+       else
+           break
+       fi
+   done
+   test_pid=`ps ax | grep ${DBADMIN_PID} | grep -v grep | grep dbAdmin.sh | wc -l`
+   if [ $test_pid -ge 1 ]; then
+       kill -9 ${DBADMIN_PID} &>/dev/null
+   fi
+   rm -f ${pidFile}
+   rm -f $CHUKWA_PID_DIR/chukwa-$CHUKWA_IDENT_STRING-dbAdmin.sh.pid
    echo "done"
 else
   echo " no $pidFile"

+ 0 - 1
src/contrib/chukwa/bin/stop-probes.sh

@@ -26,4 +26,3 @@ bin=`cd "$bin"; pwd`
 "$bin"/chukwa-daemons.sh --config $CHUKWA_CONF_DIR stop systemDataLoader.sh
 "$bin"/chukwa-daemon.sh --config $CHUKWA_CONF_DIR stop torqueDataLoader.sh
 "$bin"/chukwa-daemon.sh --config $CHUKWA_CONF_DIR stop nodeActivityDataLoader.sh
-"$bin"/chukwa-daemons.sh --config $CHUKWA_CONF_DIR --watchdog stop agent.sh

+ 44 - 9
src/contrib/chukwa/bin/systemDataLoader.sh

@@ -22,19 +22,34 @@ bin=`cd "$bin"; pwd`
 
 JVM_OPTS="-Xms4M -Xmx4M"
 
-if [ "X$1" = "Xstop" ]; then
+trap 'shutdown' 1 2 15
+
+function shutdown {
   echo -n "Shutting down System Data Loader..."
   if [ -f ${CHUKWA_HOME}/var/run/Sar-data-loader.pid ]; then
-    kill -TERM `cat ${CHUKWA_HOME}/var/run/Sar-data-loader.pid`
+    kill -9 `cat ${CHUKWA_HOME}/var/run/Sar-data-loader.pid`
   fi
   if [ -f ${CHUKWA_HOME}/var/run/Iostat-data-loader.pid ]; then
-    kill -TERM `cat ${CHUKWA_HOME}/var/run/Iostat-data-loader.pid`
+    kill -9 `cat ${CHUKWA_HOME}/var/run/Iostat-data-loader.pid`
   fi
   if [ -f ${CHUKWA_HOME}/var/run/Top-data-loader.pid ]; then
-    kill -TERM `cat ${CHUKWA_HOME}/var/run/Top-data-loader.pid`
+    kill -9 `cat ${CHUKWA_HOME}/var/run/Top-data-loader.pid`
   fi
   if [ -f ${CHUKWA_HOME}/var/run/Df-data-loader.pid ]; then
-    kill -TERM `cat ${CHUKWA_HOME}/var/run/Df-data-loader.pid`
+    kill -9 `cat ${CHUKWA_HOME}/var/run/Df-data-loader.pid`
+  fi
+  if [ -f ${CHUKWA_HOME}/var/run/Netstat-data-loader.pid ]; then
+    kill -9 `cat ${CHUKWA_HOME}/var/run/Netstat-data-loader.pid`
+  fi
+  rm -f $CHUKWA_HOME/var/run/chukwa-$CHUKWA_IDENT_STRING-systemDataLoader.sh.pid
+  echo "done"
+  exit 0
+}
+
+if [ "X$1" = "Xstop" ]; then
+  echo -n "Shutting down System Data Loader..."
+  if [ -f $CHUKWA_HOME/var/run/chukwa-$CHUKWA_IDENT_STRING-systemDataLoader.sh.pid ]; then
+    kill -TERM `head $CHUKWA_HOME/var/run/chukwa-$CHUKWA_IDENT_STRING-systemDataLoader.sh.pid`
   fi
   echo "done"
   exit 0
@@ -58,7 +73,7 @@ if [ -f $pidFile ]; then
 fi
 
 if [ ${EXISTS} -lt 1 ]; then
-    ${JAVA_HOME}/bin/java $JVM_OPTS -DPERIOD=60 -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DRECORD_TYPE=Sar -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 sar -q -r -n FULL 55 &
+    ${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=Sar -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 sar -q -r -n FULL 55 &
 fi
 
 EXISTS=0
@@ -72,7 +87,7 @@ if [ -f $pidFile ]; then
 fi
 
 if [ ${EXISTS} -lt 1 ]; then
-  ${JAVA_HOME}/bin/java $JVM_OPTS -DPERIOD=60 -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DRECORD_TYPE=Iostat -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 iostat -x 55 2 &
+  ${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=Iostat -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 iostat -x 55 2 &
 fi
 
 EXISTS=0
@@ -86,7 +101,7 @@ if [ -f $pidFile ]; then
 fi
 
 if [ ${EXISTS} -lt 1 ]; then
-  ${JAVA_HOME}/bin/java $JVM_OPTS -DPERIOD=60 -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DRECORD_TYPE=Top -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 top -b -n 1 -c &
+  ${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=Top -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 top -b -n 1 -c &
 fi
 
 EXISTS=0
@@ -100,7 +115,27 @@ if [ -f $pidFile ]; then
 fi
 
 if [ ${EXISTS} -lt 1 ]; then
-  ${JAVA_HOME}/bin/java $JVM_OPTS -DPERIOD=60 -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_DIR} -DRECORD_TYPE=Df -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 df -l &
+  ${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=Df -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 df -l &
+fi
+
+EXISTS=0
+pidFile="${CHUKWA_HOME}/var/run/Netstat-data-loader.pid"
+if [ -f $pidFile ]; then
+  pid=`head ${pidFile}`
+  ChildPIDRunningStatus=`${JPS} | grep ${pid} | grep Exec | 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=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
 
 echo "done"
+
+while [ 1 ]
+do
+    # sleep until shutdown signal has been sent.
+    sleep 5
+done

+ 2 - 2
src/contrib/chukwa/bin/torqueDataLoader.sh

@@ -41,8 +41,8 @@ if [ -f $pidFile ]; then
   pid=`head ${pidFile}`
   ChildPIDRunningStatus=`${JPS} | grep ${pid} | grep TorqueDataLoader | grep -v grep | wc -l`
   if [ $ChildPIDRunningStatus -lt 1 ]; then
-      ${java} -DDOMAIN=${DOMAIN} -DTORQUE_SERVER=${TORQUE_SERVER} -DTORQUE_HOME=${TORQUE_HOME} -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_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&
+      ${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 
 else
-      ${java} -DDOMAIN=${DOMAIN} -DTORQUE_SERVER=${TORQUE_SERVER} -DTORQUE_HOME=${TORQUE_HOME} -DCHUKWA_HOME=${CHUKWA_HOME} -DCHUKWA_CONF_DIR=${CHUKWA_CONF_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&
+      ${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

+ 48 - 0
src/contrib/chukwa/bin/validateDemux.sh

@@ -0,0 +1,48 @@
+#!/bin/sh
+# 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.
+
+pid=$$
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+. "$bin"/chukwa-config.sh
+
+echo "hadoop jar for agent is " ${HADOOP_JAR}
+now=`date +%Y%m%d_%H_%M%S`
+hdfsDir="/test_$now/"
+
+HADOOP_CMDE="${HADOOP_HOME}/bin/hadoop "
+
+$HADOOP_CMDE dfs -mkdir ${hdfsDir}
+
+echo "Moving data to HDFS: ${hdfsDir}"
+
+$HADOOP_CMDE dfs -put ${CHUKWA_HOME}/data/demuxData ${hdfsDir}/
+
+echo "demuxTestData: "
+$HADOOP_CMDE dfs -ls ${hdfsDir}/demuxData/input
+exitCode=$?
+echo "ls ExitCode: ${exitCode} "
+ 
+$HADOOP_CMDE jar  ${CHUKWA_CORE} org.apache.hadoop.chukwa.extraction.demux.Demux -Dmapred.compress.map.output=true -Dmapred.map.output.compression.codec=org.apache.hadoop.io.compress.LzoCodec -Dmapred.output.compress=true -Dmapred.output.compression.type=BLOCK -r 4 ${hdfsDir}/demuxData/input ${hdfsDir}/demuxData/output
+exitCode=$?
+echo "Demux ExitCode: ${exitCode} "
+
+${JAVA_HOME}/bin/java -Xms10M -Xmx32M -classpath /tmp/chukwaTest.jar:${CLASSPATH}:${HADOOP_JAR}:${COMMON} org.apache.hadoop.chukwa.validationframework.DemuxDirectoryValidator -hdfs ${hdfsDir}/demuxData/gold ${hdfsDir}/demuxData/output
+exitCode=$?
+echo "Validation ExitCode: ${exitCode} "
+

+ 472 - 84
src/contrib/chukwa/build.xml

@@ -19,42 +19,166 @@
 
 <project name="chukwa" default="main">
 
-  <import file="../build-contrib.xml"/>
-	<property name="chukwaVersion" value="0.1.0"/>
-
-	<property name="lib.dir" value="${basedir}/lib"/>
-	<property name="src.dir" value="${basedir}/src"/>
-        <property name="build.classes" value="${build.dir}/classes"/>
-	<property name="conf.dir" value="${basedir}/conf"/>
-	<property name="docs.dir" value="${basedir}/docs"/>
-	<property name="tools.dir" value="${basedir}/tools"/>
-	<property name="dist.dir" value="${basedir}/dist"/>
-	<property name="opt.dir" value="${basedir}/opt"/>
-        <property name="javac.debug" value="on"/>
-        <property name="javac.version" value="1.6"/>
+	<property name="name" value="chukwa"/>
+        <property name="chukwaVersion" value="0.1.1"/>
+	<property name="rpm.prefix" value="/usr/local"/>
+	<property name="hadoop.conf" value="/usr/local/hadoop/conf"/>
+	<property name="final.name" value="${name}-${chukwaVersion}"/>
+
+        <import file="../build-contrib.xml" optional="true"/>
+        <available file="../build-contrib.xml" property="present"/>
+
+        <condition property="standaloneMode">
+                <and><not><available file="../build-contrib.xml" property="present"/></not></and>
+        </condition>
+
+        <condition property="hadoopMode">
+                <and><available file="../build-contrib.xml" property="present"/></and>
+        </condition>
+
+        <target name="init-standalone" if="standaloneMode">
+          <echo>Standalone Mode</echo>
+          <property environment="env"/>
+	  <property name="basedir" value="."/>
+          <!--SET UP HADOOP JARS-->
+          <property name="hadoop.home.dir" value="${env.HADOOP_HOME}"/>
+          <echo message="HADOOP_HOME set to: ${hadoop.home.dir}"/>
+          <property name="hadoop.default.jars.dir" value="${basedir}/hadoopjars"/>
+
+          <condition property="hadoop.home.jars.dir" value="${hadoop.home.dir}/build" else="${basedir}/hadoopjars">
+                <available file="${hadoop.home.dir}/build"/>
+          </condition>
+          <echo message="hadoop.home.jars.dir set to ${hadoop.home.jars.dir}"/>
+
+          <property name="build.dir" value="${basedir}/build"/>
+          <property name="build.classes" value="${basedir}/build/classes"/>
+          <property name="test.build.dir" value="${build.dir}/test"/>
+          <property name="test.build.classes" value="${test.build.dir}/classes"/>
+          <property name="lib.dir" value="${basedir}/lib"/>
+          <path id="classpath">
+                  <fileset dir="${lib.dir}">
+                          <include name="**/*.jar" />
+                          <exclude name="**/excluded/" />
+                  </fileset>
+                  <fileset dir="${hadoop.default.jars.dir}">   <!-- ASR -->
+                          <include name="**/*.jar" />
+                          <exclude name="**/*core*.jar" />
+                  </fileset>
+                  <fileset dir="${hadoop.home.jars.dir}">   <!-- ASR -->
+                          <include name="**/*core*.jar" />
+                  </fileset>
+          </path>
+          <path id="testClasspath">
+                  <pathelement location="${build.classes}"/>
+                  <pathelement location="${test.build.classes}"/>
+                  <fileset dir="${lib.dir}">
+                          <include name="**/*.jar" />
+                          <exclude name="**/excluded/" />
+                  </fileset>
+                  <fileset dir="${hadoop.default.jars.dir}">   <!-- ASR -->
+                          <include name="**/*.jar" />
+                          <exclude name="**/*core*.jar" />
+                  </fileset>
+                  <fileset dir="${hadoop.home.jars.dir}">   <!-- ASR -->
+                          <include name="**/*core*.jar" />
+                  </fileset>
+          </path>
+          <path id="testDemuxClasspath">
+                  <pathelement location="${build.classes}"/>
+                  <pathelement location="${test.build.classes}"/>
+                  <fileset dir="${hadoop.jar}">
+                          <include name="**/*.jar" />
+                          <exclude name="**/excluded/" />
+                  </fileset>
+                  <fileset dir="${lib.dir}">
+                          <include name="**/*.jar" />
+                          <exclude name="**/excluded/" />
+                  </fileset>
+
+          </path>
+        </target>
+        
+        <target name="init-hadoop" if="hadoopMode">
+                <echo>Hadoop Mode</echo>
+	        <property name="build.dir" value="${basedir}/build"/>
+	        <property name="build.classes" value="${basedir}/build/classes"/>
+                <property name="test.build.dir" value="${build.dir}/test"/>
+                <property name="test.build.classes" value="${test.build.dir}/classes"/>
+                <delete file="${build.dir}/${final.name}/hadoop-*-core.jar" />
+	        <property name="lib.dir" value="${basedir}/lib"/>
+	        <path id="classpath">
+		        <fileset dir="${lib.dir}">
+			        <include name="**/*.jar" />
+			        <exclude name="**/excluded/" />
+		        </fileset>
+		        <fileset dir="${hadoop.root}/lib">
+			        <include name="**/*.jar" />
+			        <exclude name="**/excluded/" />
+		        </fileset>
+                        <pathelement location="${hadoop.root}/build/classes"/>
+	        </path>
+                <path id="testClasspath">
+                        <pathelement location="${build.classes}"/>
+                        <pathelement location="${test.build.classes}"/>
+                        <fileset dir="${lib.dir}">
+                                <include name="**/*.jar" />
+                                <exclude name="**/excluded/" />
+                        </fileset>
+		        <fileset dir="${hadoop.root}/lib">
+			        <include name="**/*.jar" />
+                                <exclude name="**/excluded/" />
+                        </fileset>
+                        <pathelement location="${hadoop.root}/build/classes"/>
+                        <pathelement location="${hadoop.root}/build/test/classes"/>
+                </path>
+
+                <path id="testDemuxClasspath">
+                        <pathelement location="${build.classes}"/>
+                        <pathelement location="${test.build.classes}"/>
+                        <fileset dir="${hadoop.root}/lib">
+                                <include name="**/*.jar" />
+                                <exclude name="**/excluded/" />
+                        </fileset>
+                        <fileset dir="${lib.dir}">
+                                <include name="**/*.jar" />
+                                <exclude name="**/excluded/" />
+                        </fileset>
+                        <pathelement location="${hadoop.root}/build/classes"/>
+                        <pathelement location="${hadoop.root}/build/test/classes"/>
+                </path>
        
-        <property name="test.src.dir" value="${basedir}/src/test"/>
-        <property name="test.lib.dir" value="${basedir}/src/test/lib"/>
-        <property name="test.build.dir" value="${build.dir}/test"/>
-        <property name="test.generated.dir" value="${test.build.dir}/src"/>
-        <property name="test.build.data" value="${test.build.dir}/data"/>
-        <property name="test.cache.data" value="${test.build.dir}/cache"/>
-        <property name="test.debug.data" value="${test.build.dir}/debug"/>
-        <property name="test.log.dir" value="${test.build.dir}/logs"/>
-        <property name="test.build.classes" value="${test.build.dir}/classes"/>
-        <property name="test.build.testjar" value="${test.build.dir}/testjar"/>
-        <property name="test.include" value="Test*"/>
-        <property name="test.classpath.id" value="test.classpath"/>
-        <property name="test.output" value="yes"/>
-        <property name="test.timeout" value="900000"/>
-        <property name="test.junit.output.format" value="plain"/>
-        <property name="test.junit.fork.mode" value="perTest" />
-        <property name="test.junit.printsummary" value="yes" />
-        <property name="test.junit.haltonfailure" value="yes" />
-        <property name="test.junit.maxmemory" value="256m" />
- 
-
-	<target name="init">
+
+        </target>
+
+	<target name="init" depends="init-standalone,init-hadoop">
+                <property name="src.dir" value="${basedir}/src"/>
+                <property name="build.classes" value="${build.dir}/classes"/>
+                <property name="conf.dir" value="${basedir}/conf"/>
+                <property name="docs.dir" value="${basedir}/docs"/>
+                <property name="tools.dir" value="${basedir}/tools"/>
+                <property name="dist.dir" value="${basedir}/dist"/>
+                <property name="opt.dir" value="${basedir}/opt"/>
+                <property name="javac.debug" value="on"/>
+                <property name="javac.version" value="1.6"/>
+                <property name="test.src.dir" value="${basedir}/src/test"/>
+                <property name="test.lib.dir" value="${basedir}/src/test/lib"/>
+                <property name="test.build.dir" value="${build.dir}/test"/>
+                <property name="test.generated.dir" value="${test.build.dir}/src"/>
+                <property name="test.build.data" value="${test.build.dir}/data"/>
+                <property name="test.cache.data" value="${test.build.dir}/cache"/>
+                <property name="test.debug.data" value="${test.build.dir}/debug"/>
+                <property name="test.log.dir" value="${test.build.dir}/logs"/>
+                <property name="test.build.classes" value="${test.build.dir}/classes"/>
+                <property name="test.build.testjar" value="${test.build.dir}/testjar"/>
+                <property name="test.include" value="Test*"/>
+                <property name="test.classpath.id" value="test.classpath"/>
+                <property name="test.output" value="yes"/>
+                <property name="test.timeout" value="900000"/>
+                <property name="test.junit.output.format" value="plain"/>
+                <property name="test.junit.fork.mode" value="perTest" />
+                <property name="test.junit.printsummary" value="yes" />
+                <property name="test.junit.haltonfailure" value="yes" />
+                <property name="test.junit.maxmemory" value="256m" />
 		<mkdir dir="${build.dir}"/>
 		<mkdir dir="${build.classes}"/>
 		<mkdir dir="${build.dir}/test"/>
@@ -63,7 +187,6 @@
 		</exec>
 
 		<mkdir dir="${dist.dir}"/>
-		<property name="final.name" value="${name}-${chukwaVersion}"/>
 
 	</target>
 
@@ -74,8 +197,11 @@
 	</target>
 
 	<target name="compile" depends="init" description="Compilation target">
+		<mkdir dir="${build.dir}"/>
+		<mkdir dir="${build.classes}"/>
+		<mkdir dir="${build.dir}/test"/>
 		<javac srcdir="src/java/org/apache/hadoop/chukwa" destdir="${build.classes}" excludes="**/ChukwaTTInstru.java" debug="${javac.debug}">
-			<classpath refid="chukwaHadoopClasspath" />
+			<classpath refid="classpath" />
 		</javac>
 	</target>
 
@@ -88,8 +214,9 @@
            <delete dir="${test.build.classes}"/>
            <mkdir dir="${test.build.classes}"/>
 
-            <javac srcdir="${test.src.dir}/org/apache/hadoop/chukwa" destdir="${test.build.dir}/classes" excludes="**/TestDemux.java" debug="${javac.debug}">
-                 <classpath refid="chukwaTestClasspath" />
+            <javac srcdir="${test.src.dir}/org/apache/hadoop/chukwa" destdir="${test.build.dir}/classes"  debug="${javac.debug}">
+                 <classpath refid="testClasspath" />
+                 <classpath refid="testDemuxClasspath" />
             </javac>
         </target>
 
@@ -104,7 +231,7 @@
             maxmemory="${test.junit.maxmemory}"
             dir="${test.build.dir}/classes/" timeout="${test.timeout}"
             errorProperty="tests.failed" failureProperty="tests.failed">
-           <classpath refid="chukwaTestClasspath"/>
+           <classpath refid="testClasspath"/>
            <sysproperty key="test.src.dir" value="${test.src.dir}"/>
            <formatter type="${test.junit.output.format}" />
            <batchtest todir="${test.build.dir}" >
@@ -167,6 +294,7 @@
 			<fileset dir="${build.dir}/collector" includes="**" />
 		</jar>
 	</target>
+
 	<target name="tools_jar" depends="compile, collector" description="Create tools jar">
 		<jar jarfile="${build.dir}/tools-${chukwaVersion}.jar" basedir="${build.classes}" includes="org/apache/hadoop/chukwa/inputtools/**/*.class">
 			<fileset dir="${basedir}/src/java">
@@ -221,7 +349,7 @@
                             </section>
                         </manifest>
 
-			<fileset dir="${build.dir}">
+			<fileset dir="${build.classes}">
 				<include name="org/apache/hadoop/chukwa/**/*.class"/>
 			</fileset>
 			<fileset dir="${basedir}/src/java">
@@ -235,10 +363,13 @@
 		<jar jarfile="${build.dir}/chukwa-hadoop-${chukwaVersion}-client.jar" basedir="${build.classes}" includes="org/apache/hadoop/chukwa/inputtools/log4j/**/*.class">
 			<fileset dir="${basedir}/src/java">
 				<include name="org/apache/hadoop/chukwa/inputtools/log4j/**/*.java"/>
-		        <include name="org/apache/hadoop/chukwa/datacollection/client/**/*.java"/>
-		        <include name="org/apache/hadoop/chukwa/util/**/*.java"/>
+		                <include name="org/apache/hadoop/chukwa/datacollection/client/**/*.java"/>
+		                <include name="org/apache/hadoop/chukwa/util/**/*.java"/>
 			</fileset>
-			<fileset dir="${build.dir}">
+			<fileset dir="${basedir}/conf">
+		                <include name="chukwa-hadoop-metrics-log4j.properties"/>
+			</fileset>
+			<fileset dir="${build.classes}">
 				<include name="org/apache/hadoop/chukwa/datacollection/client/**/*.class"/>
 				<include name="org/apache/hadoop/chukwa/util/**/*.class"/>
 				<include name="org/apache/hadoop/chukwa/datacollection/controller/*.class"/>
@@ -246,8 +377,90 @@
 		</jar>
 	</target>
 
-	<target name="compress" depends="compile,collector,collector_jar,tools_jar,agent_jar,chukwa_jar,chukwa-hadoop_jar" description="Compression target">
+        <target name="hicc" depends="compile, chukwa_jar, package-hadoop, package-standalone" description="Prepare hicc.war">
+                <mkdir dir="${build.dir}/hicc"/>
+                <mkdir dir="${build.dir}/hicc/WEB-INF"/>
+                <mkdir dir="${build.dir}/hicc/WEB-INF/classes"/>
+                <mkdir dir="${build.dir}/hicc/WEB-INF/lib"/>
+                <mkdir dir="${build.dir}/hicc/META-INF"/>
+                <copy todir="${build.dir}/hicc">
+                        <fileset dir="${basedir}/src/web/hicc">
+                                <include name="**" />
+                        </fileset>
+                </copy>
+                <copy todir="${build.dir}/hicc/WEB-INF/classes">
+                        <fileset dir="${build.classes}">
+                                <include name="org/apache/hadoop/chukwa/hicc/**/*.class" />
+                        </fileset>
+                        <fileset dir="${build.classes}">
+                                <include name="org/apache/hadoop/chukwa/conf/**/*.class" />
+                        </fileset>
+                        <fileset dir="${build.classes}">
+                                <include name="org/apache/hadoop/chukwa/datacollection/**/*.class" />
+                        </fileset>
+                        <fileset dir="${build.classes}">
+                                <include name="org/apache/hadoop/chukwa/extraction/engine/**/*.class" />
+                        </fileset>
+                        <fileset dir="${build.classes}">
+                                <include name="org/apache/hadoop/chukwa/inputtools/*.class" />
+                        </fileset>
+                        <fileset dir="${build.classes}">
+                                <include name="org/apache/hadoop/chukwa/util/**/*.class" />
+                        </fileset>
+
+                        <fileset dir="${basedir}/src/java">
+                                <include name="org/apache/hadoop/chukwa/hicc/**/*.java" />
+                        </fileset>
+                        <fileset dir="${basedir}/src/java">
+                                <include name="org/apache/hadoop/chukwa/conf/**/*.java" />
+                        </fileset>
+                        <fileset dir="${basedir}/src/java">
+                                <include name="org/apache/hadoop/chukwa/datacollection/**/*.java" />
+                        </fileset>
+                        <fileset dir="${basedir}/src/java">
+                                <include name="org/apache/hadoop/chukwa/extraction/engine/**/*.java" />
+                        </fileset>
+                        <fileset dir="${basedir}/src/java">
+                                <include name="org/apache/hadoop/chukwa/inputtools/*.java" />
+                        </fileset>
+                        <fileset dir="${basedir}/src/java">
+                                <include name="org/apache/hadoop/chukwa/util/**/*.java" />
+                        </fileset>
+
+                </copy>
+                <copy todir="${build.dir}/hicc/WEB-INF/lib">
+                        <fileset dir="${basedir}/lib">
+                                <include name="json.jar" />
+                                <include name="jstl.jar" />
+                                <include name="servlet.jar" />
+                                <include name="taglibs.jar" />
+                                <include name="commons-fileupload-*.jar" />
+                                <include name="commons-httpclient-*.jar" />
+                                <include name="commons-logging-adapters-*.jar" />
+                                <include name="commons-io-*.jar" />
+                                <include name="commons-logging-api-*.jar" />
+                                <include name="commons-logging.jar" />
+                                <include name="log4j-*.jar" />
+                        </fileset>
+                        <fileset dir="${build.dir}">
+                                <include name="${name}-core-${chukwaVersion}.jar" />
+                        </fileset>
+                        <fileset dir="${build.dir}/${final.name}/hadoopjars">
+                                <include name="*.jar" />
+                                <exclude name="jasper-*.jar" />
+                                <exclude name="jetty-*.jar" />
+                                <exclude name="jsp-api.jar" />
+                        </fileset>
+                </copy>
+        </target>
+
+        <target name="hicc_war" depends="compile, hicc" description="Create hicc jar">
+                <jar jarfile="${build.dir}/hicc.war" basedir="${build.dir}/hicc">
+                        <fileset dir="${build.dir}/hicc" includes="**" />
+                </jar>
+        </target>
 
+	<target name="compress" depends="compile,collector,collector_jar,hicc_war,tools_jar,agent_jar,chukwa_jar,chukwa-hadoop_jar" description="Compression target">
 		<copy todir="." includeEmptyDirs="false">
 			<fileset dir="${build.dir}">
 				<exclude name="**" />
@@ -256,7 +469,7 @@
 		</copy>
 	</target>
 
-	<target name="test" depends="compile,compile-test,test-chunk,test-input-tools" description="Automated Test Framework">
+	<target name="test" depends="compile,compile-test,test-chunk,test-input-tools,test-agent,test-database" description="Automated Test Framework">
 	</target>
 
 	<target name="test-input-tools" depends="compile,compile-test" description="Test Input Tools">
@@ -270,7 +483,7 @@
              errorProperty="tests.failed" failureProperty="tests.failed">
                <sysproperty key="CHUKWA_HOME" value="${basedir}"/>
                <sysproperty key="CHUKWA_CONF_DIR" value="${basedir}/conf"/>
-               <classpath refid="chukwaTestClasspath"/>
+               <classpath refid="testClasspath"/>
                <formatter type="${test.junit.output.format}" />
                <batchtest todir="${build.dir}/test" >
                <fileset dir="${test.src.dir}" includes="org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/*.java"/>
@@ -282,6 +495,85 @@
             <fail if="tests.failed">Tests failed!</fail>
 	</target>
 
+        <!-- ================================================================== -->
+        <!-- Clean.  Delete the build files, and their directories              -->
+        <!-- ================================================================== -->
+        <target name="clean" depends="init" description="Clean.  Delete the build files, and their directories">
+            <delete dir="${build.dir}"/>
+            <delete dir="${docs.src}/build"/>
+        </target>
+	<target name="test-agent" depends="compile,compile-test" description="Test Agent">
+            <junit showoutput="${test.output}"
+             printsummary="${test.junit.printsummary}"
+             haltonfailure="${test.junit.haltonfailure}"
+             fork="yes"
+             forkmode="${test.junit.fork.mode}"
+             maxmemory="${test.junit.maxmemory}"
+             dir="${basedir}" timeout="${test.timeout}"
+             errorProperty="tests.failed" failureProperty="tests.failed">
+               <sysproperty key="CHUKWA_HOME" value="${basedir}"/>
+               <sysproperty key="CHUKWA_CONF_DIR" value="${basedir}/conf"/>
+               <classpath refid="testClasspath"/>
+               <formatter type="${test.junit.output.format}" />
+               <batchtest todir="${build.dir}/test" >
+               <fileset dir="${test.src.dir}" includes="org/apache/hadoop/chukwa/datacollection/agent/TestAgent.java"/>
+               </batchtest>
+            </junit> 
+            <delete>
+               <fileset dir="${basedir}/var" includes="*"/>
+            </delete>
+            <fail if="tests.failed">Tests failed!</fail>
+	</target>
+
+	<target name="test-demux" depends="compile,compile-test" description="Test Demux">
+           <copy file="${basedir}/data/demuxData.tgz" todir="${test.cache.data}"/> 
+           <junit showoutput="${test.output}"
+             printsummary="${test.junit.printsummary}"
+             haltonfailure="${test.junit.haltonfailure}"
+             fork="yes"
+             forkmode="${test.junit.fork.mode}"
+             maxmemory="${test.junit.maxmemory}"
+             dir="${basedir}" timeout="${test.timeout}"
+             errorProperty="tests.failed" failureProperty="tests.failed">
+               <sysproperty key="test.build.data" value="${test.build.data}"/>
+               <sysproperty key="test.demux.data" value="${test.cache.data}"/>
+               <sysproperty key="CHUKWA_HOME" value="${basedir}"/>
+               <sysproperty key="CHUKWA_CONF_DIR" value="${basedir}/conf"/>
+               <classpath refid="testDemuxClasspath"/>
+               <formatter type="${test.junit.output.format}" />
+               <batchtest todir="${build.dir}/test" >
+               <fileset dir="${test.src.dir}" includes="org/apache/hadoop/chukwa/validationframework/TestDemux.java"/>
+               </batchtest>
+            </junit> 
+            <delete>
+               <fileset dir="${basedir}/var" includes="*"/>
+            </delete>
+            <fail if="tests.failed">Tests failed!</fail>
+	</target>
+
+	<target name="test-database" depends="compile,compile-test" description="Test Database">
+            <junit showoutput="${test.output}"
+             printsummary="${test.junit.printsummary}"
+             haltonfailure="${test.junit.haltonfailure}"
+             fork="yes"
+             forkmode="${test.junit.fork.mode}"
+             maxmemory="${test.junit.maxmemory}"
+             dir="${basedir}" timeout="${test.timeout}"
+             errorProperty="tests.failed" failureProperty="tests.failed">
+               <sysproperty key="CHUKWA_HOME" value="${basedir}"/>
+               <sysproperty key="CHUKWA_CONF_DIR" value="${basedir}/conf"/>
+               <classpath refid="testClasspath"/>
+               <formatter type="${test.junit.output.format}" />
+               <batchtest todir="${build.dir}/test" >
+               <fileset dir="${test.src.dir}" includes="org/apache/hadoop/chukwa/*.java"/>
+               </batchtest>
+            </junit> 
+            <delete>
+               <fileset dir="${basedir}/var" includes="*"/>
+            </delete>
+            <fail if="tests.failed">Tests failed!</fail>
+	</target>
+
 	<!-- ====================================================== -->
 	<!-- Macro definitions                                      -->
 	<!-- ====================================================== -->
@@ -301,7 +593,37 @@
 	<!-- ================================================================== -->
 	<!--                                                                    -->
 	<!-- ================================================================== -->
-	<target name="package" depends="compress" description="Build distribution">
+
+        <target name="package-hadoop" description="package hadoop from hadoop source" if="hadoopMode">
+		<mkdir dir="${build.dir}/${final.name}/hadoopjars"/>
+                <copy toDir="${build.dir}/${final.name}/hadoopjars">
+                        <fileset dir="${hadoop.root}/build">
+                                <include name="hadoop-*-core.jar" />
+                        </fileset>
+                        <fileset dir="${hadoop.root}/lib">
+                                <include name="commons-cli-*.jar" />
+                                <include name="commons-codec-*.jar" />
+                                <include name="commons-el.jar" />
+                                <include name="commons-httpclient-*.jar" />
+                                <include name="commons-logging-*.jar" />
+                                <include name="commons-net-*.jar" />
+                                <include name="jasper-*.jar" />
+                                <include name="jetty-*.jar" />
+                                <include name="jsp-api.jar" />
+                        </fileset>
+                </copy>
+        </target>
+
+        <target name="package-standalone" description="package hadoop from hadoop source" if="standaloneMode">
+		<mkdir dir="${build.dir}/${final.name}/hadoopjars"/>
+		<copy todir="${build.dir}/${final.name}/hadoopjars" includeEmptyDirs="false">
+			<fileset dir="${basedir}/hadoopjars">
+				<include name="**/*.jar" />
+			</fileset>
+		</copy>
+        </target>
+
+	<target name="package" depends="compress,package-hadoop,package-standalone" description="Build distribution">
 		<mkdir dir="${build.dir}/${final.name}"/>
 		<mkdir dir="${build.dir}/${final.name}/lib"/>
 		<mkdir dir="${build.dir}/${final.name}/bin"/>
@@ -330,11 +652,11 @@
 		</exec>
 
 		<copy todir="${build.dir}/${final.name}/bin">
-			<fileset dir="bin"/>
+			<fileset dir="${basedir}/bin"/>
 		</copy>
 
 		<copy todir="${build.dir}/${final.name}/conf">
-			<fileset dir="${conf.dir}"/>
+			<fileset dir="${basedir}/conf"/>
 		</copy>
 
 		<copy todir="${build.dir}/${final.name}/docs">
@@ -367,48 +689,140 @@
 
 		<copy todir="${build.dir}/${final.name}" file="build.xml"/>
 
-		<chmod perm="ugo+x" type="file" parallel="false">
-			<fileset dir="${build.dir}/${final.name}/bin">
-				<include name="*/bin/*" />
-			</fileset>
-		</chmod>
-
 	</target>
 
 	<!-- ================================================================== -->
 	<!-- Make release tarball                                               -->
 	<!-- ================================================================== -->
 	<target name="tar" depends="package" description="Make release tarball">
-		<macro_tar param.destfile="${dist.dir}/${final.name}.tar.gz">
+		<macro_tar param.destfile="${build.dir}/${final.name}.tar.gz">
 			<param.listofitems>
 				<tarfileset dir="${build.dir}" mode="664">
+					<exclude name="${final.name}/bin/*" />
 					<exclude name="${final.name}/org/*" />
 					<exclude name="${final.name}/collector/**" />
 					<exclude name="${final.name}/${final.name}/**" />
+					<exclude name="${final.name}/tools/**" />
+					<exclude name="${final.name}/hicc/*" />
 					<include name="${final.name}/**" />
 				</tarfileset>
 				<tarfileset dir="${build.dir}" mode="755">
 					<include name="${final.name}/bin/*" />
+					<include name="${final.name}/tools/**" />
 				</tarfileset>
 			</param.listofitems>
 		</macro_tar>
 	</target>
 
 	<target name="binary" depends="package" description="Make tarball without source and documentation">
-		<macro_tar param.destfile="${dist.dir}/${final.name}-bin.tar.gz">
+		<macro_tar param.destfile="${build.dir}/${final.name}-bin.tar.gz">
 			<param.listofitems>
 				<tarfileset dir="${build.dir}" mode="664">
+					<exclude name="${final.name}/bin/*" />
+					<exclude name="${final.name}/org/*" />
+					<exclude name="${final.name}/collector/**" />
+					<exclude name="${final.name}/${final.name}/**" />
+					<exclude name="${final.name}/tools/**" />
+					<exclude name="${final.name}/hicc/*" />
 					<exclude name="${final.name}/src/**" />
 					<exclude name="${final.name}/docs/**" />
 					<include name="${final.name}/**" />
 				</tarfileset>
 				<tarfileset dir="${build.dir}" mode="755">
 					<include name="${final.name}/bin/*" />
+					<include name="${final.name}/tools/**" />
 				</tarfileset>
 			</param.listofitems>
 		</macro_tar>
 	</target>
 
+        <target name="rpm" depends="clean,tar" description="Make release rpm">
+		<mkdir dir="${build.dir}/BUILD"/>
+		<mkdir dir="${build.dir}/RPMS"/>
+		<mkdir dir="${build.dir}/SOURCES"/>
+		<mkdir dir="${build.dir}/SPECS"/>
+		<copy todir="${build.dir}/SOURCES">
+			<fileset dir="${build.dir}">
+			        <include name="${final.name}.tar.gz" />
+                        </fileset>
+                </copy>
+                <echo file="${build.dir}/SPECS/chukwa.spec" append="false">
+# RPM Spec file for Chukwa v.${chukwaVersion}
+
+%define _topdir         ${build.dir}
+%define _prefix         ${rpm.prefix}
+%define _conf           ${hadoop.conf}
+%define name            chukwa
+%define summary         Distributed Computing Monitoring Framework.
+%define version         ${chukwaVersion}
+%define release         1
+%define license         ASF 2.0
+%define group           Development/Monitoring
+%define source          %{name}-%{version}.tar.gz
+%define vendor          Apache Software Fundation
+%define packager        Eric Yang
+%define buildroot       %{_topdir}/BUILD
+
+Name:      %{name}
+Version:   %{version}
+Release:   %{release}
+Packager:  %{packager}
+Vendor:    %{vendor}
+License:   %{license}
+Summary:   %{summary}
+Group:     %{group}
+Source0:   %{source}
+Prefix:    %{_prefix}
+Buildroot: %{buildroot}
+
+%description
+Chukwa is the monitoring framework for large scale distributed
+clusters.
+
+%prep
+%setup -q
+%build
+mkdir -p %{buildroot}%{_prefix}
+if [ -d %{buildroot}%{_prefix}/%{name} ]; then
+    rm -rf %{buildroot}%{_prefix}/%{name}
+fi 
+mv %{buildroot}/%{name}-%{version} %{buildroot}%{_prefix}/%{name}
+cd %{buildroot}%{_prefix}
+mkdir -p %{buildroot}/etc/init.d
+cp %{name}/tools/init.d/chukwa-data-processors %{buildroot}/etc/init.d
+cp %{name}/tools/init.d/chukwa-collector %{buildroot}/etc/init.d
+cp %{name}/tools/init.d/chukwa-agent %{buildroot}/etc/init.d
+cp %{name}/tools/init.d/chukwa-system-metrics %{buildroot}/etc/init.d
+chmod a+x %{buildroot}/etc/init.d/chukwa-*
+rm -rf %{buildroot}%{_prefix}/%{name}/src
+rm -rf %{buildroot}%{_prefix}/%{name}/build.xml
+%post
+mkdir -p %{_prefix}/%{name}
+chown -R gmon:users %{_prefix}/%{name}
+echo "Congratulation!  You have successfully installed Chukwa."
+echo ""
+echo "To collect Data             : /etc/init.d/chukwa-system-metrics start"
+echo "To run Chukwa Agent         : /etc/init.d/chukwa-agent start"
+echo "To run Chukwa Collector     : /etc/init.d/chukwa-collector start"
+echo "To run Chukwa Data Processor: /etc/init.d/chukwa-data-processors start"
+%preun
+/etc/init.d/chukwa-data-processors stop
+/etc/init.d/chukwa-system-metrics stop
+/etc/init.d/chukwa-agent stop
+/etc/init.d/chukwa-collector stop
+echo
+%postun
+%files
+%defattr(-,root,root)
+%{_prefix}/%{name}/*
+/etc/init.d/chukwa-data-processors
+/etc/init.d/chukwa-collector
+/etc/init.d/chukwa-agent
+/etc/init.d/chukwa-system-metrics
+                </echo>
+		<rpm specFile="chukwa.spec" topDir="${build.dir}" cleanBuildDir="true" failOnError="true"/>
+        </target>
+
 	<!-- the normal classpath -->
 	<path id="classpath">
 		<pathelement location="${build.classes}"/>
@@ -419,30 +833,4 @@
 		<pathelement location="${conf.dir}"/>
 	</path>
 
-	<!-- the hadoop classpath -->
-	<path id="chukwaHadoopClasspath">
-		<pathelement location="${hadoop.root}/build/classes"/>
-		<fileset dir="${lib.dir}">
-			<include name="**/*.jar" />
-			<exclude name="**/excluded/" />
-		</fileset>
-		<fileset dir="${hadoop.root}/lib">
-			<include name="**/*.jar" />
-			<exclude name="**/excluded/" />
-		</fileset>
-	</path>
-        <path id="chukwaTestClasspath">
-                <pathelement location="${hadoop.root}/build/classes"/>
-                <pathelement location="${build.classes}"/>
-                <pathelement location="${test.build.classes}"/>
-                <fileset dir="${lib.dir}">
-                        <include name="**/*.jar" />
-                        <exclude name="**/excluded/" />
-                </fileset>
-                <fileset dir="${hadoop.root}/lib">
-                        <include name="**/*.jar" />
-                        <exclude name="**/excluded/" />
-                </fileset>
-        </path>
-
 </project>

+ 0 - 24
src/contrib/chukwa/chukwafy_jobhist.awk

@@ -1,24 +0,0 @@
-#!/usr/bin/gawk
-
-#A small awk script to take normal job history logs and chukwafy
-#In particular, they get prefixed with something of the form
-#"2008-07-28 23:30:38,865 INFO org.apache.hadoop.chukwa.ChukwaJobHistory: 
-
-BEGIN {
-OFS = ""
-}
-
-{  # for each record
-  #get Timestamp
-for(i =1; i <= NF; ++i) {
-	if( $i ~ /_TIME=/) {
-		split($i, halves, "=");
-		ts_msec = substr(halves[2], 2 , length(halves[2]) - 2)
-		break;
-	}
-}
-if(ts_msec == 0)
-   print "WARNING:  no timestamp in line " > /dev/stderr
-
-print strftime("%Y-%m-%d %H:%M:%S", ts_msec/1000)  , "," , (ts_msec%1000) , " INFO org.apache.hadoop.chukwa.ChukwaJobHistory: " , $0
-}

+ 18 - 0
src/contrib/chukwa/conf/README

@@ -0,0 +1,18 @@
+Check for an updated copy of this README at http://wiki.apache.org/hadoop/Chukwa_Configuration
+
+Chukwa comes with templates for all configuration files. These files are in the conf directory and are named <conf file name>.template. You should be able to just make a copy of each of these files, removing the ".template" suffix.
+
+'''Required configuration files'''
+
+ * conf/collectors - new line delimited list of collectors. This file is used by startup and shutdown scripts to determine where to run Chukwa  
+ * collectors and is also used by Chukwa agents to find collectors to send their data to.
+ * conf/chukwa-agents - a list of hosts in the cluster on which to run the Chukwa agent daemon.
+ * conf/chukwa-env.sh - environment variables required to run Chukwa.
+ * conf/chukwa-collectors-conf.xml - collector specific settings (e.g. port number to listen for agents on, which file system to store data in, etc.)
+ * conf/chukwa-agents-conf.xml - agent specific settings.
+
+'''Optional configuration files'''
+
+ * conf/alert.conf - a list of e-mail addresses to which chukwa alerts should be sent.
+ * conf/initial_adaptors - a list of adaptors to add to all agents at startup time.
+

+ 12 - 0
src/contrib/chukwa/conf/aggregator.sql

@@ -0,0 +1,12 @@
+insert into [cluster_system_metrics] (select timestamp,[avg(system_metrics)] from [system_metrics] where timestamp between '[past_hour]' and '[now]' group by timestamp);
+insert into [dfs_throughput] (select timestamp,[avg(dfs_datanode)] from [dfs_datanode] where timestamp between '[past_hour]' and '[now]' group by timestamp);
+insert into [cluster_disk] (select a.timestamp,a.mount,a.used,a.available,a.used_percent from (select from_unixtime(unix_timestamp(timestamp)-unix_timestamp(timestamp)%60)as timestamp,mount,avg(used) as used,avg(available) as available,avg(used_percent) as used_percent from [disk] where timestamp between '[past_hour]' and '[now]' group by timestamp,mount) as a group by a.timestamp, a.mount);
+insert into [hod_job_digest] (select timestamp,d.hodid,d.userid,[avg(system_metrics)] from (select a.HodID,b.host as machine,a.userid,a.starttime,a.endtime from [HodJob] a join [hod_machine] b on (a.HodID = b.HodID) where endtime between '[past_hour]' and '[now]') as d,[system_metrics] where timestamp between d.starttime and d.endtime and host=d.machine group by hodid,timestamp);
+insert into [cluster_hadoop_rpc] (select timestamp,[avg(hadoop_rpc)] from [hadoop_rpc] where timestamp between '[past_hour]' and '[now]' group by timestamp);
+#insert into [cluster_hadoop_mapred] (select timestamp,[avg(hadoop_mapred_job)] from [hadoop_mapred_job] where timestamp between '[past_hour]' and '[now]' group by timestamp);
+insert into [user_util] (select timestamp, j.UserID as user, sum(j.NumOfMachines) as node_total, sum(cpu_idle_pcnt*j.NumOfMachines) as cpu_unused, sum((cpu_user_pcnt+cpu_system_pcnt)*j.NumOfMachines) as cpu_used, avg(cpu_user_pcnt+cpu_system_pcnt) as cpu_used_pcnt, sum((100-(sda_busy_pcnt+sdb_busy_pcnt+sdc_busy_pcnt+sdd_busy_pcnt)/4)*j.NumOfMachines) as disk_unused, sum(((sda_busy_pcnt+sdb_busy_pcnt+sdc_busy_pcnt+sdd_busy_pcnt)/4)*j.NumOfMachines) as disk_used, avg(((sda_busy_pcnt+sdb_busy_pcnt+sdc_busy_pcnt+sdd_busy_pcnt)/4)) as disk_used_pcnt, sum((100-eth0_busy_pcnt)*j.NumOfMachines) as network_unused, sum(eth0_busy_pcnt*j.NumOfMachines) as network_used, avg(eth0_busy_pcnt) as network_used_pcnt, sum((100-mem_used_pcnt)*j.NumOfMachines) as memory_unused, sum(mem_used_pcnt*j.NumOfMachines) as memory_used, avg(mem_used_pcnt) as memory_used_pcnt from [hod_job_digest] d,[HodJob] j where (d.HodID = j.HodID) and Timestamp between '[past_hour]' and '[now]' group by j.UserID);
+#insert into [node_util] select starttime, avg(unused) as unused, avg(used) as used from (select DATE_FORMAT(m.LAUNCH_TIME,'%Y-%m-%d %H:%i:%s') as starttime,sum(AvgCPUBusy*j.NumOfMachines/(60*100)) as unused,sum((100-AvgCPUBusy)*j.NumOfMachines/(60*100)) as used from HodJobDigest d join HodJob j on (d.HodID = j.HodID) join MRJob m on (m.HodID = j.HodID) where m.LAUNCH_TIME >= '2008-09-12 21:11' and m.LAUNCH_TIME <= '2008-09-12 22:11' and d.Timestamp >= m.LAUNCH_TIME and d.Timestamp <= m.FINISH_TIME group by m.MRJobID order by m.LAUNCH_TIME) as t group by t.starttime 
+#insert into [jobtype_util] select CASE WHEN MRJobName like 'PigLatin%' THEN 'Pig' WHEN MRJobName like 'streamjob%' THEN 'Streaming' WHEN MRJobName like '%abacus%' THEN 'Abacus' ELSE 'Other' END as m, count(*)*j.NumOfMachines/60 as nodehours,count(distinct(MRJobID)) as jobs from HodJobDigest d join HodJob j on (d.HodID = j.HodID) join MRJob m on (m.HodID = j.HodID) where d.Timestamp >= '2008-09-12 21:11' and d.Timestamp <= '2008-09-12 22:11' and d.Timestamp >= m.LAUNCH_TIME and d.Timestamp <= m.FINISH_TIME group by CASE WHEN MRJobName like 'PigLatin%' THEN 'Pig' WHEN MRJobName like 'streamjob%' THEN 'Streaming' WHEN MRJobName like '%abacus%' THEN 'Abacus' ELSE 'Other' END order by CASE WHEN MRJobName like 'PigLatin%' THEN 'Pig' WHEN MRJobName like 'streamjob%' THEN 'Streaming' WHEN MRJobName like '%abacus%' THEN 'Abacus' ELSE 'Other' END
+#insert into [a] select d.Timestamp as starttime,((AvgCPUBusy * j.NumOfMachines) / (sum(j.NumOfMachines) * 1)) as used from Digest d join HodJob j on (d.HodID = j.HodID) where d.Timestamp >= '[past_hour]' and d.Timestamp <= '[now]' group by d.Timestamp order by d.Timestamp 
+#insert into [b] select m, sum(foo.nodehours) as nodehours from (select m.MRJobID, round(avg(if(AvgCPUBusy is null,0,AvgCPUBusy)),0) as m, count(*)*j.NumOfMachines/60 as nodehours from HodJobDigest d join HodJob j on (d.HodID = j.HodID) join MRJob m on (m.HodID = j.HodID) where d.Timestamp >= '[past_hour]' and d.Timestamp <= '[now]' and d.Timestamp >= m.LAUNCH_TIME and d.Timestamp <= m.FINISH_TIME group by m.MRJobID) as foo group by m; 
+#insert into [c] select if(AvgCPUBusy is null,0,AvgCPUBusy) as m, CASE WHEN MRJobName like 'PigLatin%' THEN 'Pig' WHEN MRJobName like 'streamjob%' THEN 'Streaming' WHEN MRJobName like '%abacus%' THEN 'Abacus' ELSE 'Other' END as interface, count(*)*j.NumOfMachines/60 as nodehours,count(distinct(MRJobID)) as jobs from HodJobDigest d join HodJob j on (d.HodID = j.HodID) join MRJob m on (m.HodID = j.HodID) where d.Timestamp >= '[past_hour]' and d.Timestamp <= '[now]' and d.Timestamp >= m.LAUNCH_TIME and d.Timestamp <= m.FINISH_TIME group by AvgCPUBusy,CASE WHEN MRJobName like 'PigLatin%' THEN 'Pig' WHEN MRJobName like 'streamjob%' THEN 'Streaming' WHEN MRJobName like '%abacus%' THEN 'Abacus' ELSE 'Other' END order by if(AvgCPUBusy is null,0,AvgCPUBusy)

+ 8 - 7
src/contrib/chukwa/conf/chukwa-agent-conf.xml.template

@@ -5,6 +5,13 @@
 
 <configuration>
   
+  
+  <property>
+    <name>chukwaAgent.tags</name>
+    <value>cluster="demo"</value>
+    <description>The cluster's name for this agent</description>
+  </property>
+  
   <property>
     <name>chukwaAgent.control.port</name>
     <value>9093</value>
@@ -17,12 +24,6 @@
     <description>The hostname of the agent on this node. Usually localhost, this is used by the chukwa instrumentation agent-control interface library</description>
   </property>
 
-  <property>
-    <name>chukwaAgent.tags</name>
-    <value>cluster="demo"</value>
-    <description>The cluster's name for this agent</description>
-  </property>
-
   <property>
     <name>chukwaAgent.checkpoint.name</name>
     <value>chukwa_agent_checkpoint</value>
@@ -31,7 +32,7 @@
   
   <property>
     <name>chukwaAgent.checkpoint.dir</name>
-    <value>/tmp/</value>
+    <value>${CHUKWA_HOME}/var/tmp/</value>
     <description>the location to put the agent's checkpoint file(s)</description>
   </property>
 

+ 0 - 0
src/contrib/chukwa/conf/chukwa-slaves.template → src/contrib/chukwa/conf/chukwa-agents.template


+ 107 - 0
src/contrib/chukwa/conf/chukwa-demux-conf.xml

@@ -0,0 +1,107 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="nutch-conf.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+
+  <property>
+    <name>chukwaArchiveBuilder.reduceCount</name>
+    <value>5</value>
+    <description>Reduce count </description>
+  </property>
+
+  
+  <property>
+    <name>SysLog</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.SysLog</value>
+    <description>Parser class for </description>
+  </property>
+
+  <property>
+    <name>Df</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.Df</value>
+    <description>Parser class for </description>
+  </property>
+
+  <property>
+    <name>HadoopLog</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.HadoopLogProcessor</value>
+    <description>Parser class for </description>
+  </property>
+
+  <property>
+    <name>HadoopMetricsProcessor</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.HadoopMetricsProcessor</value>
+    <description>Parser class for </description>
+  </property>
+
+  <property>
+    <name>Iostat</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.Iostat</value>
+    <description>Parser class for </description>
+  </property>
+ 
+   <property>
+    <name>Log4jJobHistoryProcessor</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.Log4jJobHistoryProcessor</value>
+    <description>Parser class for </description>
+  </property>
+
+   <property>
+    <name>JobLogHistoryProcessor</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.JobLogHistoryProcessor</value>
+    <description>Parser class for </description>
+  </property>
+
+ 
+   <property>
+    <name>PbsNodes</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.PbsNodes</value>
+    <description>Parser class for </description>
+  </property>
+ 
+   <property>
+    <name>Sar</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.Sar</value>
+    <description>Parser class for </description>
+  </property>
+
+   <property>
+    <name>TsProcessor</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.TsProcessor</value>
+    <description>Parser class for </description>
+   </property>
+  
+   <property>
+    <name>Top</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.Top</value>
+    <description>Parser class for </description>
+   </property>
+
+   <property>
+    <name>Torque</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.Torque</value>
+    <description>Parser class for Parsing qstat and tracejob</description>
+   </property>
+  
+   <property>
+    <name>YWatch</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.YWatch</value>
+    <description>Parser class for </description>
+   </property>
+  
+   <property>
+    <name>DbLoader</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.TsProcessor</value>
+    <description>Parser class for </description>
+   </property>
+
+   <property>
+    <name>JobConf</name>
+    <value>org.apache.hadoop.chukwa.extraction.demux.processor.mapper.JobConfProcessor</value>
+    <description>Parser class for Map reduce Job Configuration</description>
+   </property>
+    
+</configuration>

+ 46 - 20
src/contrib/chukwa/conf/chukwa-env.sh.template

@@ -6,35 +6,61 @@
 # remote nodes.
 
 # The java implementation to use.  Required.
-export JAVA_HOME=/usr/lib/j2sdk1.5-sun
+#export JAVA_HOME=/usr/lib/j2sdk1.5-sun
 
-# The location of the Hadoop the collector should use. Default 
-# assumes that this chukwa is living in hadoop's src/contrib directory
-export HADOOP_HOME="/usr/lib/hadoop/current"
+# Optional (.ie. will try sensible defaults)
+# The location of the Hadoop the collector should use. Chukwa uses this to 
+# find the hadoop classes or jars as well as the hadoop executables
+# for running the mapreduce demux job. Chukwa will look first in
+# HADOOP_HOME/build for a hadoop-*-core.jar file, 
+# then in HADOOP_HOME/build/classes for hadoop classes.
+# If you want to use a hadoop jar straight up, without
+# a HADOOP_HOME/bin directory, you should set HADOOP_JAR (below), but
+# then chukwa mapreduce jobs wont work. If HADOOP_HOME is not set, default
+# assumes that this chukwa is living in hadoop's src/contrib directory, but
+# if jars or classes can not be found using that default assumption, 
+# Chukwa will default to the hadoop jars that come with it
+# in the hadoopjars directory. 
+#export HADOOP_HOME="/home/user/Development/hadoop-trunk"
 
-# The directory where pid files are stored. CHUKWA_HOME/var/run by default.
-#export CHUKWA_PID_DIR=
+# Optional (i.e. will try sensible defaults)
+# The location of a hadoop jars. Setting this will overrided the search for 
+# hadoop jars described above (using HADOOP_HOME). If HADOOP_JAR is not set, 
+# the default is to check HADOOP_HOME/build (above) for 
+# jars or classes, if those are not found, uses hadoop jars which
+# come with chukwa in $CHUKWA_HOME/hadoopjars. YOU SHOULD ONLY NEED TO 
+# USE THIS IF YOU ARE RUNNING A COLLECTOR WITHOUT A RUNNING 
+# HDFS! (i.e. writing datasink sequence files to local disk). Be careful 
+# if you use HDFS for chukwa storage, but haven't built the 
+# hadoop classes or jar because chukwa will use the default hadoop jars 
+# that come with ckukwa, you can easily suffer
+# errors due to protocol mismatch between hadoop versions.
+#export HADOOP_JAR=${HADOOP_HOME}/build/hadoop-*-core.jar
 
-# The location of chukwa logs, defaults to CHUKWA_HOME/logs
-export CHUKWA_LOG_DIR=${CHUKWA_HOME}/logs
+# The location of chukwa data repository (in either HDFS or your local
+# file system, whichever you are using)
+export chukwaRecordsRepository="/chukwa/repos/"
 
-# The location of a hadoop jars. use this if you are running a collector
-# without a running HDFS (i.e. which writes sequence files to local disk)
-# if this is not set, the default is to check HADOOP_HOME for jars or
-# classes, if those are not found, uses hadoop jars which come with chukwa
-export HADOOP_JAR=`ls ${HADOOP_HOME}/hadoop-*-core.jar`
+# The directory where pid files are stored. CHUKWA_HOME/var/run by default.
+#export CHUKWA_PID_DIR="/tmp/chukwa-pid-dir"
 
-# The location of chukwa data repository
-export chuwaRecordsRepository="/chukwa/repos/"
+# The location of chukwa logs, defaults to CHUKWA_HOME/logs
+#export CHUKWA_LOG_DIR="/tmp/chukwa-log-dir"
 
 # The location of torque pbsnodes command
-export nodeActivityCmde="/usr/lib/torque/current/bin/pbsnodes "
+#export nodeActivityCmde="
 
 # The server which contain pbsnodes, qstat and tracejob.
-export TORQUE_SERVER=localhost
+#export TORQUE_SERVER=localhost
 
 # The location contain torque binaries.
-export TORQUE_HOME=/usr/lib/torque
+#export TORQUE_HOME=/usr/local/torque
+
+# Instance name for chukwa deployment
+export CHUKWA_IDENT_STRING=demo
+
+# Datatbase driver name for storing Chukwa Data.
+# export JDBC_DRIVER=
 
-# The domain of the cluster
-#export DOMAIN=
+# Database URL prefix for Database Loader.
+# export JDBC_URL_PREFIX=jdbc://

+ 31 - 0
src/contrib/chukwa/conf/chukwa-hadoop-metrics-log4j.properties

@@ -0,0 +1,31 @@
+log4j.appender.chukwa.rpc.recordType=HadoopMetricsProcessor
+log4j.appender.chukwa.rpc.chukwaClientHostname=localhost
+log4j.appender.chukwa.rpc.chukwaClientPortNum=9093
+log4j.appender.chukwa.rpc.DatePattern=.yyyy-MM-dd
+log4j.appender.chukwa.rpc.layout=org.apache.log4j.PatternLayout
+log4j.appender.chukwa.rpc.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.appender.chukwa.rpc.Dir=/tmp
+
+log4j.appender.chukwa.jvm.recordType=HadoopMetricsProcessor
+log4j.appender.chukwa.jvm.chukwaClientHostname=localhost
+log4j.appender.chukwa.jvm.chukwaClientPortNum=9093
+log4j.appender.chukwa.jvm.DatePattern=.yyyy-MM-dd
+log4j.appender.chukwa.jvm.layout=org.apache.log4j.PatternLayout
+log4j.appender.chukwa.jvm.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.appender.chukwa.jvm.Dir=/tmp
+
+log4j.appender.chukwa.dfs.recordType=HadoopMetricsProcessor
+log4j.appender.chukwa.dfs.chukwaClientHostname=localhost
+log4j.appender.chukwa.dfs.chukwaClientPortNum=9093
+log4j.appender.chukwa.dfs.DatePattern=.yyyy-MM-dd
+log4j.appender.chukwa.dfs.layout=org.apache.log4j.PatternLayout
+log4j.appender.chukwa.dfs.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.appender.chukwa.dfs.Dir=/tmp
+
+log4j.appender.chukwa.mapred.recordType=HadoopMetricsProcessor
+log4j.appender.chukwa.mapred.chukwaClientHostname=localhost
+log4j.appender.chukwa.mapred.chukwaClientPortNum=9093
+log4j.appender.chukwa.mapred.DatePattern=.yyyy-MM-dd
+log4j.appender.chukwa.mapred.layout=org.apache.log4j.PatternLayout
+log4j.appender.chukwa.mapred.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.appender.chukwa.mapred.Dir=/tmp

+ 19 - 0
src/contrib/chukwa/conf/chukwa-log4j.properties

@@ -0,0 +1,19 @@
+log4j.rootLogger=INFO, R 
+log4j.appender.R=org.apache.log4j.RollingFileAppender
+log4j.appender.R.File=${CHUKWA_LOG_DIR}/${APP}.log
+log4j.appender.R.MaxFileSize=10MB
+log4j.appender.R.MaxBackupIndex=10
+log4j.appender.R.layout=org.apache.log4j.PatternLayout
+log4j.appender.R.layout.ConversionPattern=%d{ISO8601} %p %t %c{1} - %m%n
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this 
+#
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
+
+

+ 1 - 1
src/contrib/chukwa/conf/collectors.template

@@ -1 +1 @@
-http://localhost:8080/
+localhost

+ 602 - 0
src/contrib/chukwa/conf/database_create_tables

@@ -0,0 +1,602 @@
+create table if not exists node_activity_template 
+(
+    timestamp  timestamp default CURRENT_TIMESTAMP,
+    used int(11) default NULL,
+    usedMachines text,
+    free int(11) default NULL,
+    freeMachines text,
+    down int(11) default NULL,
+    downMachines text,
+    primary key(timestamp),
+    index (Timestamp)
+);
+
+create table if not exists switch_data_template (
+    timestamp timestamp default CURRENT_TIMESTAMP,
+    host varchar(40),
+    port varchar(10),
+    poller varchar(40),
+    metricName varchar(20),
+    value double,
+    primary key(timestamp, host, port),
+    index (Timestamp)
+);
+
+create table if not exists system_metrics_template (
+    timestamp  timestamp default CURRENT_TIMESTAMP,
+    host varchar(40),
+    load_15 double, 
+    load_5 double,
+    load_1 double,
+    task_total double,
+    task_running double,
+    task_sleep double,
+    task_stopped double,
+    task_zombie double,
+    mem_total double,
+    mem_buffers double,
+    mem_cached double,
+    mem_used double,
+    mem_free double,
+    eth0_rxerrs double,
+    eth0_rxbyts double,
+    eth0_rxpcks double,
+    eth0_rxdrops double,
+    eth0_txerrs double,
+    eth0_txbyts double,
+    eth0_txpcks double,
+    eth0_txdrops double,
+    eth1_rxerrs double,
+    eth1_rxbyts double,
+    eth1_rxpcks double,
+    eth1_rxdrops double,
+    eth1_txerrs double,
+    eth1_txbyts double,
+    eth1_txpcks double,
+    eth1_txdrops double,
+    sda_rkbs double,
+    sda_wkbs double,
+    sdb_rkbs double,
+    sdb_wkbs double,
+    sdc_rkbs double,
+    sdc_wkbs double,
+    sdd_rkbs double,
+    sdd_wkbs double,
+    cpu_idle_pcnt float,
+    cpu_nice_pcnt float,
+    cpu_system_pcnt float,
+    cpu_user_pcnt float,
+    cpu_hirq_pcnt float,
+    cpu_sirq_pcnt float,
+    iowait_pcnt float,
+    mem_buffers_pcnt float,
+    mem_used_pcnt float,
+    eth0_busy_pcnt float,
+    eth1_busy_pcnt float,
+    sda_busy_pcnt float,
+    sdb_busy_pcnt float,
+    sdc_busy_pcnt float,
+    sdd_busy_pcnt float,
+    swap_used_pcnt float,
+    primary key(host, timestamp),
+    index (timestamp)
+);
+
+create table if not exists disk_template (
+    timestamp  timestamp default CURRENT_TIMESTAMP,
+    host varchar(40),
+    mount varchar(40),
+    used double,
+    available double,
+    used_percent double,
+    fs varchar(40),
+    primary key(timestamp,host,mount),
+    index (timestamp)
+);
+
+create table if not exists cluster_disk_template (
+    timestamp  timestamp default CURRENT_TIMESTAMP,
+    mount varchar(40),
+    used double,
+    available double,
+    used_percent double,
+    primary key(timestamp,mount),
+    index (timestamp)
+);
+
+create table if not exists cluster_system_metrics_template (
+    timestamp  timestamp default CURRENT_TIMESTAMP,
+    host int,
+    load_15 double, 
+    load_5 double,
+    load_1 double,
+    task_total double,
+    task_running double,
+    task_sleep double,
+    task_stopped double,
+    task_zombie double,
+    mem_total double,
+    mem_buffers double,
+    mem_cached double,
+    mem_used double,
+    mem_free double,
+    eth0_rxerrs double,
+    eth0_rxbyts double,
+    eth0_rxpcks double,
+    eth0_rxdrops double,
+    eth0_txerrs double,
+    eth0_txbyts double,
+    eth0_txpcks double,
+    eth0_txdrops double,
+    eth1_rxerrs double,
+    eth1_rxbyts double,
+    eth1_rxpcks double,
+    eth1_rxdrops double,
+    eth1_txerrs double,
+    eth1_txbyts double,
+    eth1_txpcks double,
+    eth1_txdrops double,
+    sda_rkbs double,
+    sda_wkbs double,
+    sdb_rkbs double,
+    sdb_wkbs double,
+    sdc_rkbs double,
+    sdc_wkbs double,
+    sdd_rkbs double,
+    sdd_wkbs double,
+    cpu_idle_pcnt float,
+    cpu_nice_pcnt float,
+    cpu_system_pcnt float,
+    cpu_user_pcnt float,
+    cpu_hirq_pcnt float,
+    cpu_sirq_pcnt float,
+    iowait_pcnt float,
+    mem_buffers_pcnt float,
+    mem_used_pcnt float,
+    eth0_busy_pcnt float,
+    eth1_busy_pcnt float,
+    sda_busy_pcnt float,
+    sdb_busy_pcnt float,
+    sdc_busy_pcnt float,
+    sdd_busy_pcnt float,
+    swap_used_pcnt float,
+    primary key(host, timestamp),
+    index (timestamp)
+);
+
+create table if not exists dfs_namenode_template (
+    timestamp timestamp default 0,
+    host varchar(80),
+    add_block_ops double,
+    blocks_corrupted double,
+    create_file_ops double,
+    delete_file_ops double,
+    files_created double,
+    files_renamed double,
+    files_deleted double,
+    get_block_locations double,
+    get_listing_ops double,
+    safe_mode_time double,
+    syncs_avg_time double,
+    syncs_num_ops double,
+    transactions_avg_time double,
+    transactions_num_ops double,
+    block_report_avg_time double,
+    block_report_num_ops double,
+    fs_image_load_time double,
+    primary key(timestamp, host),
+    index(timeStamp)
+);
+
+create table if not exists dfs_datanode_template (
+    timestamp timestamp default 0,
+    host varchar(80),
+    block_reports_avg_time double,
+    block_reports_num_ops double,
+    block_verification_failures double,
+    blocks_read double,
+    blocks_removed double,
+    blocks_replicated double,
+    blocks_verified double,
+    blocks_written double,
+    bytes_read double,
+    bytes_written double,
+    copy_block_op_avg_time double,
+    copy_block_op_num_ops double,
+    heart_beats_avg_time double,
+    heart_beats_num_ops double,
+    read_block_op_avg_time double,
+    read_block_op_num_ops double,
+    read_metadata_op_avg_time double,
+    read_metadata_op_num_ops double,
+    reads_from_local_client double,
+    reads_from_remote_client double,
+    replace_block_op_avg_time double,
+    replace_block_op_num_ops double,
+    session_id double,
+    write_block_op_avg_time double,
+    write_block_op_num_ops double,
+    writes_from_local_client double,
+    writes_from_remote_client double,
+    primary key(timestamp, host),
+    index(timestamp)
+);
+
+create table if not exists dfs_fsnamesystem_template (
+    timestamp timestamp default 0,
+    host VARCHAR(80),
+    blocks_total double,
+    capacity_remaining_gb double,
+    capacity_total_gb double,
+    capacity_used_gb double,
+    files_total double,
+    pending_replication_blocks double,
+    scheduled_replication_blocks double,
+    total_load double,
+    under_replicated_blocks double,
+    primary key(timestamp, host),
+    index(timestamp)
+);
+
+create table if not exists dfs_throughput_template (
+    timestamp timestamp default 0,
+    host int,
+    block_reports_avg_time double,
+    block_reports_num_ops double,
+    block_verification_failures double,
+    blocks_read double,
+    blocks_removed double,
+    blocks_replicated double,
+    blocks_verified double,
+    blocks_written double,
+    bytes_read double,
+    bytes_written double,
+    copy_block_op_avg_time double,
+    copy_block_op_num_ops double,
+    heart_beats_avg_time double,
+    heart_beats_num_ops double,
+    read_block_op_avg_time double,
+    read_block_op_num_ops double,
+    read_metadata_op_avg_time double,
+    read_metadata_op_num_ops double,
+    reads_from_local_client double,
+    reads_from_remote_client double,
+    replace_block_op_avg_time double,
+    replace_block_op_num_ops double,
+    session_id double,
+    write_block_op_avg_time double,
+    write_block_op_num_ops double,
+    writes_from_local_client double,
+    writes_from_remote_client double,
+    primary key(timestamp),
+    index(timestamp)
+);
+
+create table if not exists hadoop_jvm_template (
+    timestamp timestamp default 0,
+    host VARCHAR(80),
+    process_name VARCHAR(80),
+    gc_timemillis double,
+    gc_count double,
+    log_error double,
+    log_fatal double,
+    log_info double,
+    log_warn double,
+    mem_heap_committed_m double,
+    mem_heap_used_m double,
+    mem_non_heap_committed_m double,
+    mem_non_heap_used_m double,
+    threads_blocked double,
+    threads_new double,
+    threads_runnable double,
+    threads_terminated double,
+    threads_timed_waiting double,
+    threads_waiting double,
+    primary key (timestamp, host, process_name),
+    index(timestamp)
+);
+
+create table if not exists hadoop_mapred_template (
+    timestamp timestamp default 0,
+    host VARCHAR(80),
+    jobs_completed double,
+    jobs_submitted double,
+    maps_completed double,
+    maps_launched double,
+    reduces_completed double,
+    reduces_launched double,
+    primary key (timestamp, host),
+    index(timestamp)
+);
+
+create table if not exists hadoop_rpc_template (
+    timestamp timestamp default 0,
+    host VARCHAR(80),
+    rpc_processing_time_avg_time double,
+    rpc_processing_time_num_ops double,
+    rpc_queue_time_avg_time double,
+    rpc_queue_time_num_ops double,
+    get_build_version_avg_time double,
+    get_build_version_num_ops double,
+    get_job_counters_avg_time double,
+    get_job_counters_num_ops double,
+    get_job_profile_avg_time double,
+    get_job_profile_num_ops double,
+    get_job_status_avg_time double,
+    get_job_status_num_ops double,
+    get_new_job_id_avg_time double,
+    get_new_job_id_num_ops double,
+    get_protocol_version_avg_time double,
+    get_protocol_version_num_ops double,
+    get_system_dir_avg_time double,
+    get_system_dir_num_ops double,
+    get_task_completion_events_avg_time double,
+    get_task_completion_events_num_ops double,
+    get_task_diagnostics_avg_time double,
+    get_task_diagnostics_num_ops double,
+    heartbeat_avg_time double,
+    heartbeat_num_ops double,
+    killJob_avg_time double,
+    killJob_num_ops double,
+    submit_job_avg_time double,
+    submit_job_num_ops double,
+    primary key (timestamp, host),
+    index(timestamp)
+);
+
+create table if not exists cluster_hadoop_rpc_template (
+    timestamp timestamp default 0,
+    host int,
+    rpc_processing_time_avg_time double,
+    rpc_processing_time_num_ops double,
+    rpc_queue_time_avg_time double,
+    rpc_queue_time_num_ops double,
+    get_build_version_avg_time double,
+    get_build_version_num_ops double,
+    get_job_counters_avg_time double,
+    get_job_counters_num_ops double,
+    get_job_profile_avg_time double,
+    get_job_profile_num_ops double,
+    get_job_status_avg_time double,
+    get_job_status_num_ops double,
+    get_new_job_id_avg_time double,
+    get_new_job_id_num_ops double,
+    get_protocol_version_avg_time double,
+    get_protocol_version_num_ops double,
+    get_system_dir_avg_time double,
+    get_system_dir_num_ops double,
+    get_task_completion_events_avg_time double,
+    get_task_completion_events_num_ops double,
+    get_task_diagnostics_avg_time double,
+    get_task_diagnostics_num_ops double,
+    heartbeat_avg_time double,
+    heartbeat_num_ops double,
+    killJob_avg_time double,
+    killJob_num_ops double,
+    submit_job_avg_time double,
+    submit_job_num_ops double,
+    primary key (timestamp),
+    index(timestamp)
+);
+
+create table if not exists hadoop_rpc_calls_template (
+    timestamp timestamp default 0,
+    method varchar(80),
+    calls double,
+    primary key(timestamp, method),
+    index(timestamp)
+);
+
+create table if not exists mssrgraph_template (
+    timestamp timestamp default 0,
+    job_id VARCHAR(80),
+    type VARCHAR(10),
+    count double,
+    primary key (timestamp, job_id),
+    index(timestamp)
+);
+
+create table if not exists mr_job(
+    HodID varchar(20),
+    MRJobID varchar(80),
+    MRJobName varchar(100),
+    STATUS varchar(10),
+    SUBMIT_TIME timestamp default 0,
+    LAUNCH_TIME timestamp default 0,
+    FINISH_TIME timestamp default 0,
+    MAPPER_PHASE_END_TIME timestamp default 0,
+    TOTAL_MAPS int unsigned,
+    TOTAL_REDUCES int unsigned,
+    FINISHED_MAPS  int unsigned,
+    FINISHED_REDUCES  int unsigned,      
+    NumOfLocalSplits int unsigned,
+    NumOfRackLocalSplits int unsigned,
+    NUM_OF_MAPPER_ATTEMPTS int unsigned,
+    NUM_OF_REDUCER_ATTEMPTS int unsigned,
+    MAPPER_PHASE_EXECUTION_TIME int,
+    AvgMapperExecutionTime int unsigned,
+    AvgLocalMapperExecutionTime int unsigned,
+    AvgRackLocalMapperExecutionTime int unsigned,
+    AvgRemoteMapperExecutionTime int unsigned,
+    AvgReducerExecutionTime int unsigned,
+    AvgShuffleExecutionTime int unsigned,
+    AvgSortExecutionTime int unsigned,
+    MapperClass varchar(80),
+    ReducerClass varchar(80),
+    PartitionerClass varchar(80),
+    CombinerClass varchar(80),
+    InputFormatClass varchar(80),
+    InputKeyClass varchar(80),
+    InputValueClass varchar(80),
+    OutputFormatClass varchar(80),
+    OutputCompressed tinyint,
+    OutputCompressionType  varchar(20),
+    OutputCompressionCodec varchar(20),
+    OutputKeyClass varchar(80),
+    OutputValueClass varchar(80),
+    MapOutputKeyClass varchar(80),
+    MapOutputValueClass varchar(80),
+    MapOutputCompressed tinyint,
+    MapOutputCompressionType  varchar(20),
+    MapOutputCompressionCodec varchar(20),
+    InputDataSizeInMB int unsigned,
+    MapOutputSizeInMB int unsigned,
+    OutputDataSizeInMB int unsigned,
+    MR_JOBCONF  text, 
+    InputDir  text, 
+    primary key(SUBMIT_TIME, HodID, MRJobID),
+    index(SUBMIT_TIME)
+);
+
+create table if not exists mr_job_counters_template (
+    timestamp timestamp default 0,
+    job_id BIGINT,
+    hdfs_bytes_read double,
+    hdfs_bytes_written double,
+    local_bytes_read double,
+    local_bytes_written double,
+    data_local_map_tasks double,
+    launched_map_tasks double,
+    launched_reduce_tasks double,
+    combine_input_records double,
+    combine_output_records double,
+    map_input_bytes double,
+    map_output_bytes double,
+    map_input_records double,
+    map_output_records double,
+    reduce_input_groups double,
+    reduce_input_records double,
+    reduce_output_records double,
+    primary key (timestamp, job_id),
+    index(timestamp)
+);
+
+create table if not exists mr_job_timeline_template (
+    timestamp timestamp default CURRENT_TIMESTAMP,
+    mr_job_id varchar(80),
+    task_type varchar(20),
+    tasks double,
+    time double,
+    primary key(timestamp, mr_job_id),
+    index(timestamp)
+);
+
+create table if not exists mr_finish_time_vs_size_template (
+    timestamp timestamp default CURRENT_TIMESTAMP,
+    mr_job_id varchar(80),
+    task_type varchar(20),
+    size double,
+    time double,
+    primary key(timestamp, mr_job_id),
+    index(timestamp)
+);
+    
+create table if not exists hod_machine_template (
+    hodid varchar(20) not null, 
+    host varchar(40) not null,
+    index(HodId)
+);
+
+create table if not exists HodJob_template (
+    HodID varchar(20), 
+    UserID varchar(20), 
+    Status  smallint,
+    JobTracker varchar(40), 
+    TimeQueued mediumint unsigned,
+    StartTime timestamp default CURRENT_TIMESTAMP, 
+    EndTime timestamp default 0,  
+    NumOfMachines smallint unsigned,  
+    SlotLimitPerTracker smallint unsigned,
+    LogProcessStatus varchar(20),
+    primary key(HodId),
+    index(StartTime, EndTime)
+);
+
+create table if not exists hod_job_digest_template (
+    timestamp timestamp default 0,
+    HodID VARCHAR(20),
+    UserID VARCHAR(20),
+    host int,
+    load_15 double, 
+    load_5 double,
+    load_1 double,
+    task_total double,
+    task_running double,
+    task_sleep double,
+    task_stopped double,
+    task_zombie double,
+    mem_total double,
+    mem_buffers double,
+    mem_cached double,
+    mem_used double,
+    mem_free double,
+    eth0_rxerrs double,
+    eth0_rxbyts double,
+    eth0_rxpcks double,
+    eth0_rxdrops double,
+    eth0_txerrs double,
+    eth0_txbyts double,
+    eth0_txpcks double,
+    eth0_txdrops double,
+    eth1_rxerrs double,
+    eth1_rxbyts double,
+    eth1_rxpcks double,
+    eth1_rxdrops double,
+    eth1_txerrs double,
+    eth1_txbyts double,
+    eth1_txpcks double,
+    eth1_txdrops double,
+    sda_rkbs double,
+    sda_wkbs double,
+    sdb_rkbs double,
+    sdb_wkbs double,
+    sdc_rkbs double,
+    sdc_wkbs double,
+    sdd_rkbs double,
+    sdd_wkbs double,
+    cpu_idle_pcnt float,
+    cpu_nice_pcnt float,
+    cpu_system_pcnt float,
+    cpu_user_pcnt float,
+    cpu_hirq_pcnt float,
+    cpu_sirq_pcnt float,
+    iowait_pcnt float,
+    mem_buffers_pcnt float,
+    mem_used_pcnt float,
+    eth0_busy_pcnt float,
+    eth1_busy_pcnt float,
+    sda_busy_pcnt float,
+    sdb_busy_pcnt float,
+    sdc_busy_pcnt float,
+    sdd_busy_pcnt float,
+    swap_used_pcnt float,
+    primary key(HodId, timestamp),
+    index(timeStamp)
+); 
+
+create table if not exists user_util_template (
+    timestamp timestamp default CURRENT_TIMESTAMP,
+    user VARCHAR(20),
+    node_total int,
+    cpu_unused double,
+    cpu_used double,
+    cpu_used_pcnt float,
+    disk_unused double,
+    disk_used double,
+    disk_used_pcnt float,
+    network_unused double,
+    network_used double,
+    network_used_pcnt float,
+    memory_unused double,
+    memory_used double,
+    memory_used_pcnt float
+);
+
+create table if not exists QueueInfo(
+    Timestamp timestamp default 0,
+    HodID VARCHAR(20),
+    Queue VARCHAR(20),
+    NumOfMachine smallint unsigned,
+    status varchar(1),
+    index(TimeStamp)
+);

+ 25 - 26
src/contrib/chukwa/hadoop-packaging/log4j.properties.templ → src/contrib/chukwa/conf/hadoop-log4j.properties

@@ -1,48 +1,31 @@
 # Define some default values that can be overridden by system properties
-hadoop.root.logger=DEBUG,console
+hadoop.root.logger=INFO,console
 hadoop.log.dir=.
 hadoop.log.file=hadoop.log
 
 # Define the root logger to the system property "hadoop.root.logger".
-log4j.rootLogger=${hadoop.root.logger},DRFA,EventCounter
+log4j.rootLogger=${hadoop.root.logger}, EventCounter
 
 # Logging Threshold
 log4j.threshhold=ALL
 
-
-
-log4j.logger.org.apache.hadoop.chukwa.ChukwaJobHistory=INFO, jobhistory
-log4j.appender.jobhistory=org.apache.hadoop.chukwa.inputtools.log4j.ChukwaDailyRollingFileAppender
-log4j.appender.jobhistory.File=${hadoop.log.dir}/JobLogHistory.log
-
-log4j.appender.jobhistory.recordType=JobLogHistoryProcessor
-log4j.appender.jobhistory.chukwaClientHostname=localhost
-log4j.appender.jobhistory.chukwaClientPortNum=9093
-
-# Rollver at midnight
-log4j.appender.jobhistory.DatePattern=.yyyy-MM-dd
-
-# 30-day backup
-#log4j.appender.DRFA.MaxBackupIndex=30
-log4j.appender.jobhistory.layout=org.apache.log4j.PatternLayout
-
-# Pattern format: Date LogLevel LoggerName LogMessage
-log4j.appender.jobhistory.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
-
-
-
 #
 # Daily Rolling File Appender
 #
 
 #log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
-log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
+#log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
 
+#
+# CHUKWA
+#
 log4j.appender.DRFA=org.apache.hadoop.chukwa.inputtools.log4j.ChukwaDailyRollingFileAppender
-log4j.appender.DRFA.recordType=HadoopLogProcessor
+log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
+log4j.appender.DRFA.recordType=HadoopLog
 log4j.appender.DRFA.chukwaClientHostname=localhost
 log4j.appender.DRFA.chukwaClientPortNum=9093
 
+
 # Rollver at midnight
 log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
 
@@ -55,6 +38,21 @@ log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
 # Debugging Pattern format
 #log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
 
+#
+# AUDIT LOGGING - All audit events are logged at INFO level
+#
+# CHUKWA AUDIT LOG
+
+log4j.appender.DRFAAUDIT=org.apache.hadoop.chukwa.inputtools.log4j.ChukwaDailyRollingFileAppender
+log4j.appender.DRFAAUDIT.File=${hadoop.log.dir}/audit.log
+log4j.appender.DRFAAUDIT.recordType=HadoopLog
+log4j.appender.DRFAAUDIT.chukwaClientHostname=localhost
+log4j.appender.DRFAAUDIT.chukwaClientPortNum=9093
+log4j.appender.DRFAAUDIT.DatePattern=.yyyy-MM-dd
+log4j.appender.DRFAAUDIT.layout=org.apache.log4j.PatternLayout
+log4j.appender.DRFAAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.logger.org.apache.hadoop.fs.FSNamesystem.audit=INFO,DRFAAUDIT
+log4j.additivity.org.apache.hadoop.fs.FSNamesystem.audit=false
 
 #
 # console
@@ -110,3 +108,4 @@ log4j.appender.TLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
 # Sends counts of logging messages at different severity levels to Hadoop Metrics.
 #
 log4j.appender.EventCounter=org.apache.hadoop.metrics.jvm.EventCounter
+

+ 4 - 0
src/contrib/chukwa/conf/hadoop-metrics.properties

@@ -4,4 +4,8 @@ dfs.period=60
 jvm.class=org.apache.hadoop.chukwa.inputtools.log4j.Log4JMetricsContext
 jvm.period=60
 
+mapred.class=org.apache.hadoop.chukwa.inputtools.log4j.Log4JMetricsContext
+mapred.period=60
 
+rpc.class=org.apache.hadoop.chukwa.inputtools.log4j.Log4JMetricsContext
+rpc.period=60

+ 1 - 0
src/contrib/chukwa/conf/jdbc.conf

@@ -0,0 +1 @@
+unknown=jdbc:://localhost:3306/demo?user=example

+ 1 - 0
src/contrib/chukwa/conf/jdbc.conf.template

@@ -0,0 +1 @@
+unknown=jdbc:://localhost:3306/demo?user=example

+ 2 - 13
src/contrib/chukwa/conf/log4j.properties

@@ -1,6 +1,6 @@
-log4j.rootLogger=INFO, stdout, R
+log4j.rootLogger=INFO, R
 log4j.appender.R=org.apache.log4j.RollingFileAppender
-log4j.appender.R.File=${CHUKWA_HOME}/logs/chukwa.log
+log4j.appender.R.File=${CHUKWA_LOG_DIR}/chukwa.log
 log4j.appender.R.MaxFileSize=10MB
 log4j.appender.R.MaxBackupIndex=10
 log4j.appender.R.layout=org.apache.log4j.PatternLayout
@@ -10,14 +10,3 @@ log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %p %t %c{1} - %m%n
 
-log4j.logger.org.apache.hadoop.chukwa.ikit.DataConfig=DEBUG, R
-
-# In our hadoop log4j.properties, replace the following line:
-# log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
-# with this line:
-#log4j.appender.DRFA=org.apache.hadoop.chukwa.inputtools.log4j.DailyRollingFileAppender
-
-# Then add these lines as well:
-#log4j.appender.DRFA.recordType=hadoop_log
-#log4j.appender.DRFA.chukwaClientHostname=localhost
-#log4j.appender.DRFA.chukwaClientPortNum=9094

+ 0 - 1097
src/contrib/chukwa/conf/mdl.xml

@@ -1,1097 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="nutch-conf.xsl"?>
-
-<!-- Put site-specific property overrides in this file. -->
-
-<configuration>
-
-<!--- collector properties -->
-<property>
-  <name>chukwa.engine.dsDirectory.rootFolder</name>
-  <value>hdfs://localhost:9000/chukwa/</value>
-  <description>root folder for Chukwa HDFS storage</description>
-</property>
-
-<!--- database properties -->
-<property>
-  <name>jdbc.host</name>
-  <value>localhost:3306</value>
-  <description>mysql server hostname and port</description>
-</property>
-
-<property>
-  <name>jdbc.user</name>
-  <value>gmetrics</value>
-  <description> mysql user name</description>
-</property>
-
-<property>
-  <name>jdbc.password</name>
-  <value>gmetrics</value>
-  <description>mysql password</description>
-</property>
-
-<property>
-  <name>jdbc.db</name>
-  <value>flubbergold</value>
-  <description>name of the database</description>
-</property>
-
-<!-- Torque Data Loader Config -->
-<property>
-  <name>torque.server</name>
-  <value>hostname</value>
-</property>
-
-<property>
-  <name>torque.binDir</name>
-  <value>/grid/0/torque/current/bin</value>
-</property>
-
-<!-- Log Data Loader Config -->
-<property>
-  <name>job.log.dfs.config</name>
-  <value>/grid/0/hadoop/conf/hadoop-site.xml</value>
-</property>
-
-<property>
-  <name>job.log.dfs.log.suffix</name>
-  <value>host.example.com</value>
-</property>
-
-<property>
-  <name>job.log.dfs.path</name>
-  <value>/user</value>
-</property>
-
-<property>
-  <name>job.log.dfs.path.append.uid</name>
-  <value>on</value>
-</property>
-
-<!--- NodeActivity properties -->
-<property>
-  <name>mdl.plugin.NodeActivityPlugin.cmde</name>
-  <value>/grid/0/torque/current/bin/pbsnodes </value>
-  <description>Command to retrieve the node activity raw data used/free/down</description>
-</property>
-
-<!-- database tables -->
-<property>
-  <name>report.db.name.disks.perdisk</name>
-  <value>SimonData</value>
-  <description></description>
-</property>
-
-<property>
-  <name>report.db.name.systemstate.pernode</name>
-  <value>SimonData</value>
-</property>
-
-<property>
-  <name>report.db.name.systemstate.percluster</name>
-  <value>cluster_systemstate</value>
-</property>
-
-<property>
-  <name>report.db.name.jvm.tmp</name>
-  <value>JVM</value>
-</property>
-
-<property>
-  <name>report.db.name.dfs.hdfs throughput</name>
-  <value>dfs_throughput</value>
-</property>
-
-<property>
-  <name>report.db.name.dfs.individual datanode throughput</name>
-  <value>dfs_individual</value>
-</property>
-
-<property>
-  <name>report.db.name.dfs.namenode operations</name>
-  <value>dfs_namenode</value>
-</property>
-
-<property>
-  <name>report.db.name.dfs.fsnamesystem status</name>
-  <value>dfs_fsnamesystem_status</value>
-</property>
-
-<property>
-  <name>report.db.name.rpcmetrics.rpc metrics</name>
-  <value>rpc_metrics</value>
-</property>
-
-<property>
-  <name>report.db.name.hodjob</name>
-  <value>HodJob</value>
-</property>
-
-<property>
-  <name>report.db.name.hodmachine</name>
-  <value>HodMachine</value>
-</property>
-
-<property>
-  <name>report.db.name.mrjob</name>
-  <value>MRJob</value>
-</property>
-
-<property>
-  <name>report.db.name.mrjobts</name>
-  <value>MRJobTSData</value>
-</property>
-
-<property>
-  <name>report.db.name.hodjobunprocessed</name>
-  <value>HodJobUnprocessed</value>
-</property>
-
-<property>
-  <name>report.db.name.hodjobdigest</name>
-  <value>HodJobDigest</value>
-</property>
-
-<property>
-  <name>report.db.name.queueInfo</name>
-  <value>QueueInfo</value>
-</property>
-
-<property>
-  <name>report.db.name.jobcounters</name>
-  <value>MRJobCounters</value>
-</property>
-
-<!-- Simon Data Loader Config -->
-<property>
-  <name>normalize.disks.perdisk.device</name>
-  <value>1</value>
-</property>
-
-<property>
-  <name>report.db.primary.key.systemstate.percluster</name>
-  <value>timestamp</value>
-</property>
-
-<property>
-  <name>report.db.primary.key.disks.perdisk</name>
-  <value>timestamp</value>
-</property>
-
-<property>
-  <name>report.db.primary.key.systemstate.pernode</name>
-  <value>timestamp</value>
-</property>
-
-<property>
-  <name>report.db.primary.key.dfs.hdfs throughput</name>
-  <value>timestamp</value>
-</property>
-
-<property>
-  <name>report.db.primary.key.dfs.individual datanode throughput</name>
-  <value>timestamp</value>
-</property>
-
-<property>
-  <name>report.db.primary.key.dfs.namenode operations</name>
-  <value>timestamp</value>
-</property>
-
-<property>
-  <name>report.db.primary.key.dfs.fsnamesystem status</name>
-  <value>timestamp</value>
-</property>
-
-<property>
-  <name>report.db.primary.key.rpcmetrics.rpc metrics</name>
-  <value>timestamp</value>
-</property>
-
-<property>
-  <name>metric.disks.perdisk.node</name>
-  <value>machine</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node</name>
-  <value>machine</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.cpu_busy%</name>
-  <value>CPUBusy</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.mem_free%</name>
-  <value>FreeMemPercentage</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.cpu_idle%</name>
-  <value>CPUIdle</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.cpu_nice%</name>
-  <value>CPUNice</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.cpu_system%</name>
-  <value>CPUSystem</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.cpu_user%</name>
-  <value>CPUUser</value>
-</property>
-
-<property>V
-  <name>metric.systemstate.pernode.node.cpu_iowait%</name>
-  <value>CPUIOWait</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.cpu_irq%</name>
-  <value>CPUIrq</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.cpu_softirq%</name>
-  <value>CPUSoftIrq</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.load_fifteen</name>
-  <value>LoadFifteen</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.load_five</name>
-  <value>LoadFive</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.load_one</name>
-  <value>LoadOne</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.mem_buffers%</name>
-  <value>MemBuffersPercentage</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.mem_cached%</name>
-  <value>MemCachedPercentage</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.mem_user%</name>
-  <value>MemUserPercentage</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.mem_shared%</name>
-  <value>MemSharedPercentage</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.disk_max_busy%</name>
-  <value>MaxDiskBusyPercentage</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.bytes_in</name>
-  <value>NetworkInKBps</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.bytes_out</name>
-  <value>NetworkOutKBps</value>
-</property>
-
-<property>
-  <name>metric.disks.perdisk.node.device.sda.rKb/s</name>
-  <value>DiskAReadKBps</value>
-</property>
-
-<property>
-  <name>metric.disks.perdisk.node.device.sda.wKb/s</name>
-  <value>DiskAWriteKBps</value>
-</property>
-
-<property>
-  <name>metric.disks.perdisk.node.device.sdb.rKb/s</name>
-  <value>DiskBReadKBps</value>
-</property>
-
-<property>
-  <name>metric.disks.perdisk.node.device.sdb.wKb/s</name>
-  <value>DiskBWriteKBps</value>
-</property>
-
-<property>
-  <name>metric.disks.perdisk.node.device.sdc.rKb/s</name>
-  <value>DiskCReadKBps</value>
-</property>
-
-<property>
-  <name>metric.disks.perdisk.node.device.sdc.wKb/s</name>
-  <value>DiskCWriteKBps</value>
-</property>
-
-<property>
-  <name>metric.disks.perdisk.node.device.sdd.rKb/s</name>
-  <value>DiskDReadKBps</value>
-</property>
-
-<property>
-  <name>metric.disks.perdisk.node.device.sdd.wKb/s</name>
-  <value>DiskDWriteKBps</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.disk_total</name>
-  <value>DiskTotalGB</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.disk_used%</name>
-  <value>DiskUsedPercentage</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.pkts_in</name>
-  <value>PacketsIn</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.pkts_out</name>
-  <value>PacketsOut</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.swap_in_kBps</name>
-  <value>SwapInKBps</value>
-</property>
-
-<property>
-  <name>metric.systemstate.pernode.node.swap_out_kBps</name>
-  <value>SwapOutKBps</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..#nodes</name>
-  <value>nodes</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..#pids</name>
-  <value>pids</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgBlurbs</name>
-  <value>avgBlurbs</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgBlurbRate%</name>
-  <value>avgBlurbRate</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgCpuIdle%</name>
-  <value>avgCpuIdle</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgCpuNice</name>
-  <value>avgCpuNice</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgCpuNice%</name>
-  <value>avgCpuNicePercent</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgCpuSystem%</name>
-  <value>avgCpuSystem</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgCpuUser%</name>
-  <value>avgCpuUser</value>
-</property>
- 
-<property>
-  <name>metric.systemstate.percluster..avgCpuIowait%</name>
-  <value>avgCpuIowait</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgCpuIrq%</name>
-  <value>avgCpuIrq</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgCpuSoftirq%</name>
-  <value>avgCpuSoftirq</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgCpuBusy%</name>
-  <value>avgCpuBusy</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgLoadOne</name>
-  <value>avgLoadOne</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgLoadFive</name>
-  <value>avgLoadFive</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgLoadFifteen</name>
-  <value>avgLoadFifteen</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgMemBuffers%</name>
-  <value>avgMemBuffers</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgMemCached</name>
-  <value>avgMemCached</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgMemCached%</name>
-  <value>avgMemCachedPercent</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgMemFree</name>
-  <value>avgMemFree</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgMemFree%</name>
-  <value>avgMemFreePercent</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgMemUser</name>
-  <value>avgMemUser</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgMemUser%</name>
-  <value>avgMemUserPercent</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgMemShared</name>
-  <value>avgMemShared</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgMemShared%</name>
-  <value>avgMemSharedPercent</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgMemTotal</name>
-  <value>avgMemTotal</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgSwapTotal</name>
-  <value>avgSwapTotal</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgSwapInKbps</name>
-  <value>avgSwapInKbps</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgSwapOutKbps</name>
-  <value>avgSwapOutKbps</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgBytesIn</name>
-  <value>avgBytesIn</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgBytesOut</name>
-  <value>avgBytesOut</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgPktsIn</name>
-  <value>avgPktsIn</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgPktsOut</name>
-  <value>avgPktsOut</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgDiskFree</name>
-  <value>avgDiskFree</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgDiskTotal</name>
-  <value>avgDiskTotal</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgDiskUsed%</name>
-  <value>avgDiskUsed</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgPartMaxUsed%</name>
-  <value>avgPartMaxUsed</value>
-</property>
-
-<property>
-  <name>metric.systemstate.percluster..avgDiskMaxBusy%</name>
-  <value>avgDiskMaxBusy</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..#nodes</name>
-  <value>nodes</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..#pids</name>
-  <value>pids</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..bytesWrittenPerSec</name>
-  <value>bytesWrittenPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..blocksRemovedPerSec</name>
-  <value>blocksRemovedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..readsFromRemoteClientPerSec</name>
-  <value>readsFromLocalRemotePerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..writesFromLocalClientPerSec</name>
-  <value>writesFromLocalClientPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..blocksVerifiedPerSec</name>
-  <value>blocksVerifiedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..blocksWrittenPerSec</name>
-  <value>blocksWrittenPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..blockVerificationFailuresPerSec</name>
-  <value>blockVerificationFailuresPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..#restarts</name>
-  <value>restarts</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..blocksReplicatedPerSec</name>
-  <value>blocksReplicatedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..bytesReadPerSec</name>
-  <value>bytesReadPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..writesFromRemoteClientPerSec</name>
-  <value>writesFromLocalRemotePerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..readsFromLocalClientPerSec</name>
-  <value>readsFromLocalClientPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..blocksReadPerSec</name>
-  <value>blocksReadPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..readBlockOperationsPerSec</name>
-  <value>readBlockOperationsPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..numberReadBlockOperations</name>
-  <value>numberReadBlockOperations</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..writeBlockOperationsPerSec</name>
-  <value>writeBlockOperationsPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..numberWriteBlockOperations</name>
-  <value>numberWriteBlockOperations</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..readMetadataOperationsPerSec</name>
-  <value>readMetadataOperationsPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..numberReadMetadataOperations</name>
-  <value>numberReadMetadataOperations</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..copyBlockOperationsPerSec</name>
-  <value>copyBlockOperationsPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..numberCopyBlockOperations</name>
-  <value>numberCopyBlockOperations</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..replaceBlockOperationsPerSec</name>
-  <value>replaceBlockOperationsPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..numberReplaceBlockOperations</name>
-  <value>numberReplaceBlockOperations</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..heartBeatsPerSec</name>
-  <value>heartBeatsPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..numberHeartBeats</name>
-  <value>numberHeartBeats</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..blockReportsPerSec</name>
-  <value>blockReportsPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.hdfs throughput..numberBlockReports</name>
-  <value>numberBlockReports</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.#pids</name>
-  <value>pids</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.bytesWrittenPerSec</name>
-  <value>bytesWrittenPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.blocksRemovedPerSec</name>
-  <value>blocksRemovedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.readsFromRemoteClientPerSec</name>
-  <value>readsFromLocalRemotePerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.writesFromLocalClientPerSec</name>
-  <value>writesFromLocalClientPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.blocksVerifiedPerSec</name>
-  <value>blocksVerifiedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.blocksWrittenPerSec</name>
-  <value>blocksWrittenPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.blockVerificationFailuresPerSec</name>
-  <value>blockVerificationFailuresPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node</name>
-  <value>node</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.#restarts</name>
-  <value>restarts</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.blocksReplicatedPerSec</name>
-  <value>blocksReplicatedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.bytesReadPerSec</name>
-  <value>bytesReadPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.writesFromRemoteClientPerSec</name>
-  <value>writesFromLocalRemotePerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.readsFromLocalClientPerSec</name>
-  <value>readsFromLocalClientPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.individual datanode throughput.node.blocksReadPerSec</name>
-  <value>blocksReadPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node</name>
-  <value>node</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.#pids</name>
-  <value>pids</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.#restarts</name>
-  <value>restarts</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.filesDeletedPerSec</name>
-  <value>filesDeletedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.filesCreatedPerSec</name>
-  <value>filesCreatedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.filesOpenedPerSec</name>
-  <value>filesOpenedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.filesRenamedPerSec</name>
-  <value>filesRenamedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.filesListedPerSec</name>
-  <value>filesListedPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.numberOfTransactionsPerSec</name>
-  <value>numberOfTransactionsPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.AverageTransactions</name>
-  <value>AverageTransactions</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.timeInSafeMode</name>
-  <value>timeInSafeMode</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.numberOfSyncsPerSec</name>
-  <value>numberOfSyncsPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.AverageSyncTime</name>
-  <value>AverageSyncTime</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.fsImageLoadTime</name>
-  <value>fsImageLoadTime</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.BlocksCorrupted</name>
-  <value>BlocksCorrupted</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.numberOfBlockReportPerSec</name>
-  <value>numberOfBlockReportPerSec</value>
-</property>
-
-<property>
-  <name>metric.dfs.namenode operations.node.AverageBlockReportTime</name>
-  <value>AverageBlockReportTime</value>
-</property>
-
-<property>
-  <name>metric.dfs.fsnamesystem status.node</name>
-  <value>node</value>
-</property>
-
-<property>
-  <name>metric.dfs.fsnamesystem status.node.FilesTotal</name>
-  <value>FilesTotal</value>
-</property>
-
-<property>
-  <name>metric.dfs.fsnamesystem status.node.BlocksTotal</name>
-  <value>BlocksTotal</value>
-</property>
-
-<property>
-  <name>metric.dfs.fsnamesystem status.node.CapacityTotalGB</name>
-  <value>CapacityTotalGB</value>
-</property>
-
-<property>
-  <name>metric.dfs.fsnamesystem status.node.CapacityUsedGB</name>
-  <value>CapacityUsedGB</value>
-</property>
-
-<property>
-  <name>metric.dfs.fsnamesystem status.node.CapacityRemainingGB</name>
-  <value>CapacityRemainingGB</value>
-</property>
-
-<property>
-  <name>metric.dfs.fsnamesystem status.node.TotalLoad</name>
-  <value></value>
-</property>
-
-<property>
-  <name>metric.dfs.fsnamesystem status.node.PendingReplicationBlocks</name>
-  <value>PendingReplicationBlocks</value>
-</property>
-
-<property>
-  <name>metric.dfs.fsnamesystem status.node.UnderReplicatedBlocks</name>
-  <value>UnderReplicatedBlocks</value>
-</property>
-
-<property>
-  <name>metric.dfs.fsnamesystem status.node.ScheduledReplicationBlocks</name>
-  <value>ScheduledReplicationBlocks</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..#nodes</name>
-  <value>nodes</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..AverageRpcQueueTime_num_ops</name>
-  <value>AverageRpcQueueTime_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..AverageRpcQueueTime_avg_time</name>
-  <value>AverageRpcQueueTime_avg_time</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..AverageRpcQueueTime_num_ops</name>
-  <value>AverageRpcQueueTime_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..AverageRpcProcessingTime_num_ops</name>
-  <value>AverageRpcProcessingTime_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..AverageRpcProcessingTime_avg_time</name>
-  <value>AverageRpcProcessingTime_avg_time</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..RpcDiscarded_num_ops</name>
-  <value>RpcDiscarded_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..RpcDiscarded_avg_time</name>
-  <value>RpcDiscarded_avg_time</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..register_num_ops</name>
-  <value>register_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..register_avg_time</name>
-  <value>register_avg_time</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..getProtocolVersion_num_ops</name>
-  <value>getProtocolVersion_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..getProtocolVersion_avg_time</name>
-  <value>getProtocolVersion_avg_time</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..sendHeartbeat_num_ops</name>
-  <value>sendHeartbeat_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..sendHeartbeat_avg_time</name>
-  <value>sendHeartbeat_avg_time</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..blockReport_num_ops</name>
-  <value>blockReport_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..blockReport_avg_time</name>
-  <value>blockReport_avg_time</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..getBlockLocations_num_ops</name>
-  <value>getBlockLocations_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..heartbeat_num_ops</name>
-  <value>heartbeat_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..versionRequest_num_ops</name>
-  <value>versionRequest_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..setPermission_num_ops</name>
-  <value>setPermission_num_ops</value>
-</property>
-
-<property>
-  <name>metric.rpcmetrics.rpc metrics..rollFsImage_num_ops</name>
-  <value>rollFsImage_num_ops</value>
-</property>
-
-<property>
-  <name>conversion.metric.systemstate.pernode.node.bytes_in</name>
-  <value>0.001</value>
-</property>
-
-<property>
-  <name>conversion.metric.systemstate.pernode.node.bytes_out</name>
-  <value>0.001</value>
-</property>
-
-<property>
-  <name>util.perfMetrics</name>
-  <value>CPUBusy,FreeMemPercentage,MaxDiskBusyPercentage,NetworkInKBps,NetworkOutKBps,DiskAReadKBps,DiskBReadKBps,DiskCReadKBps,DiskDReadKBps,DiskAWriteKBps,DiskBWriteKBps,DiskCWriteKBps,DiskDWriteKBps,DiskUsedPercentage</value>
-</property>
-
-<!-- Database summarization intervals -->
-<property>
-  <name>consolidator.table.dfs_namenode</name>
-  <value>5,30,120</value>
-</property>
-
-<property>
-  <name>consolidator.table.SimonData</name>
-  <value>5,30,120</value>
-</property>
-
-<property>
-  <name>consolidator.table.rpc_metrics</name>
-  <value>5,30,120</value>
-</property>
-
-<property>
-  <name>consolidator.table.dfs_throughput</name>
-  <value>5,30,120</value>
-</property>
-
-<property>
-  <name>consolidator.table.dfs_individual</name>
-  <value>5,30,120</value>
-</property>
-
-<property>
-  <name>consolidator.table.cluster_systemstate</name>
-  <value>5,30,120</value>
-</property>
-
-<property>
-  <name>consolidator.table.NodeActivity</name>
-  <value>5,30,120</value>
-</property>
-
-<property>
-  <name>consolidator.table.HodJobDigest</name>
-  <value>5,30,120</value>
-</property>
-
-<property>
-  <name>consolidator.table.dfs_fsnamesystem_status</name>
-  <value>5,30,120</value>
-</property>
-</configuration>

+ 610 - 444
src/contrib/chukwa/conf/mdl.xml.template

@@ -4,123 +4,129 @@
 <!-- Put site-specific property overrides in this file. -->
 
 <configuration>
+<!-- database tables -->
 
-<!--- collector properties -->
 <property>
-  <name>chukwa.engine.dsDirectory.rootFolder</name>
-  <value>hdfs://localhost:9000/chukwa/</value>
-  <description>root folder for Chukwa HDFS storage</description>
+  <name>report.db.name.nodeactivity</name>
+  <value>node_activity</value>
+  <description></description>
+</property>
+  
+<property>
+  <name>report.db.primary.key.nodeactivity</name>
+  <value>timestamp</value>
 </property>
 
-<!--- database properties -->
 <property>
-  <name>jdbc.host</name>
-  <value>localhost:3306</value>
-  <description>mysql server hostname and port</description>
+  <name>metric.nodeactivity.down</name>
+  <value>down</value>
 </property>
 
 <property>
-  <name>jdbc.user</name>
-  <value>gmetrics</value>
-  <description> mysql user name</description>
+  <name>metric.nodeactivity.downmachines</name>
+  <value>downMachines</value>
 </property>
 
 <property>
-  <name>jdbc.password</name>
-  <value>gmetrics</value>
-  <description>mysql password</description>
+  <name>metric.nodeactivity.free</name>
+  <value>free</value>
 </property>
 
 <property>
-  <name>jdbc.db</name>
-  <value>flubbergold</value>
-  <description>name of the database</description>
+  <name>metric.nodeactivity.freemachines</name>
+  <value>freeMachines</value>
 </property>
 
-<!-- Torque Data Loader Config -->
 <property>
-  <name>torque.server</name>
-  <value>hostname</value>
+  <name>metric.nodeactivity.used</name>
+  <value>used</value>
 </property>
 
 <property>
-  <name>torque.binDir</name>
-  <value>/grid/0/torque/current/bin</value>
+  <name>metric.nodeactivity.usedmachines</name>
+  <value>usedMachines</value>
 </property>
 
-<!-- Log Data Loader Config -->
+
 <property>
-  <name>job.log.dfs.config</name>
-  <value>/grid/0/hadoop/conf/hadoop-site.xml</value>
+  <name>report.db.name.hod_job_digest</name>
+  <value>hod_job_digest</value>
 </property>
 
 <property>
-  <name>job.log.dfs.log.suffix</name>
-  <value>host.example.com</value>
+  <name>report.db.name.cluster_system_metrics</name>
+  <value>cluster_system_metrics</value>
 </property>
 
 <property>
-  <name>job.log.dfs.path</name>
-  <value>/user</value>
+  <name>report.db.name.systemmetrics</name>
+  <value>system_metrics</value>
+  <description></description>
 </property>
 
 <property>
-  <name>job.log.dfs.path.append.uid</name>
-  <value>on</value>
+  <name>report.db.name.df</name>
+  <value>disk</value>
 </property>
 
-<!--- NodeActivity properties -->
 <property>
-  <name>mdl.plugin.NodeActivityPlugin.cmde</name>
-  <value>/grid/0/torque/current/bin/pbsnodes </value>
-  <description>Command to retrieve the node activity raw data used/free/down</description>
+  <name>report.db.name.cluster_disk</name>
+  <value>cluster_disk</value>
 </property>
 
-<!-- database tables -->
 <property>
-  <name>report.db.name.disks.perdisk</name>
-  <value>SimonData</value>
-  <description></description>
+  <name>report.db.name.hadoop_dfs_namenode</name>
+  <value>dfs_namenode</value>
 </property>
 
 <property>
-  <name>report.db.name.systemstate.pernode</name>
-  <value>SimonData</value>
+  <name>report.db.name.hadoop_dfs_datanode</name>
+  <value>dfs_datanode</value>
 </property>
 
 <property>
-  <name>report.db.name.systemstate.percluster</name>
-  <value>cluster_systemstate</value>
+  <name>report.db.name.hadoop_dfs_throughput</name>
+  <value>dfs_throughput</value>
 </property>
 
 <property>
-  <name>report.db.name.jvm.tmp</name>
-  <value>JVM</value>
+  <name>report.db.name.hadoop_dfs_fsnamesystem</name>
+  <value>dfs_fsnamesystem</value>
 </property>
 
 <property>
-  <name>report.db.name.dfs.hdfs throughput</name>
-  <value>dfs_throughput</value>
+  <name>report.db.name.hadoop_dfs_fsdirectory</name>
+  <value>dfs_namenode</value>
 </property>
 
 <property>
-  <name>report.db.name.dfs.individual datanode throughput</name>
-  <value>dfs_individual</value>
+  <name>report.db.name.hadoop_jvm_metrics</name>
+  <value>hadoop_jvm</value>
 </property>
 
 <property>
-  <name>report.db.name.dfs.namenode operations</name>
-  <value>dfs_namenode</value>
+  <name>report.db.name.hadoop_mapred_jobtracker</name>
+  <value>hadoop_mapred</value>
+</property>
+
+<property>
+  <name>report.db.name.hadoop_rpc_metrics</name>
+  <value>hadoop_rpc</value>
+</property>
+
+<property>
+  <name>report.db.name.cluster_hadoop_rpc</name>
+  <value>cluster_hadoop_rpc</value>
 </property>
 
 <property>
-  <name>report.db.name.dfs.fsnamesystem status</name>
-  <value>dfs_fsnamesystem_status</value>
+  <name>report.db.name.mssrgraph</name>
+  <value>mssrgraph</value>
 </property>
 
 <property>
-  <name>report.db.name.rpcmetrics.rpc metrics</name>
-  <value>rpc_metrics</value>
+  <name>report.db.name.mrjobcounters</name>
+  <value>MRJobCounters</value>
 </property>
 
 <property>
@@ -130,7 +136,7 @@
 
 <property>
   <name>report.db.name.hodmachine</name>
-  <value>HodMachine</value>
+  <value>hod_machine</value>
 </property>
 
 <property>
@@ -163,935 +169,1095 @@
   <value>MRJobCounters</value>
 </property>
 
-<!-- Simon Data Loader Config -->
 <property>
-  <name>normalize.disks.perdisk.device</name>
-  <value>1</value>
+  <name>report.db.name.user_util</name>
+  <value>user_util</value>
 </property>
 
+<!-- System Metrics Config -->
 <property>
-  <name>report.db.primary.key.systemstate.percluster</name>
+  <name>report.db.primary.key.systemmetrics</name>
   <value>timestamp</value>
 </property>
 
 <property>
-  <name>report.db.primary.key.disks.perdisk</name>
-  <value>timestamp</value>
+  <name>metric.systemmetrics.csource</name>
+  <value>host</value>
 </property>
 
 <property>
-  <name>report.db.primary.key.systemstate.pernode</name>
-  <value>timestamp</value>
+  <name>metric.systemmetrics.ldavg-1</name>
+  <value>load_1</value>
 </property>
 
 <property>
-  <name>report.db.primary.key.dfs.hdfs throughput</name>
-  <value>timestamp</value>
+  <name>metric.systemmetrics.ldavg-5</name>
+  <value>load_5</value>
 </property>
 
 <property>
-  <name>report.db.primary.key.dfs.individual datanode throughput</name>
-  <value>timestamp</value>
+  <name>metric.systemmetrics.ldavg-15</name>
+  <value>load_15</value>
 </property>
 
 <property>
-  <name>report.db.primary.key.dfs.namenode operations</name>
-  <value>timestamp</value>
+  <name>metric.systemmetrics.tasks_total</name>
+  <value>task_total</value>
 </property>
-
 <property>
-  <name>report.db.primary.key.dfs.fsnamesystem status</name>
-  <value>timestamp</value>
+  <name>metric.systemmetrics.tasks_running</name>
+  <value>task_running</value>
 </property>
 
 <property>
-  <name>report.db.primary.key.rpcmetrics.rpc metrics</name>
-  <value>timestamp</value>
+  <name>metric.systemmetrics.tasks_sleeping</name>
+  <value>task_sleep</value>
 </property>
 
 <property>
-  <name>metric.disks.perdisk.node</name>
-  <value>machine</value>
+  <name>metric.systemmetrics.tasks_stopped</name>
+  <value>task_stopped</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node</name>
-  <value>machine</value>
+  <name>metric.systemmetrics.tasks_zombie</name>
+  <value>task_zombie</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.cpu_busy%</name>
-  <value>CPUBusy</value>
+  <name>metric.systemmetrics.mem_total</name>
+  <value>mem_total</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.mem_free%</name>
-  <value>FreeMemPercentage</value>
+  <name>metric.systemmetrics.mem_buffers</name>
+  <value>mem_buffers</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.cpu_idle%</name>
-  <value>CPUIdle</value>
+  <name>metric.systemmetrics.mem_free</name>
+  <value>mem_free</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.cpu_nice%</name>
-  <value>CPUNice</value>
+  <name>metric.systemmetrics.mem_used</name>
+  <value>mem_used</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.cpu_system%</name>
-  <value>CPUSystem</value>
+  <name>metric.systemmetrics.mem_shared</name>
+  <value>mem_shared</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.cpu_user%</name>
-  <value>CPUUser</value>
+  <name>metric.systemmetrics.kbcached</name>
+  <value>mem_cached</value>
 </property>
 
-<property>V
-  <name>metric.systemstate.pernode.node.cpu_iowait%</name>
-  <value>CPUIOWait</value>
+<property>
+  <name>metric.systemmetrics.eth0.rxerr/s</name>
+  <value>eth0_rxerrs</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.cpu_irq%</name>
-  <value>CPUIrq</value>
+  <name>metric.systemmetrics.eth0.rxbyt/s</name>
+  <value>eth0_rxbyts</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.cpu_softirq%</name>
-  <value>CPUSoftIrq</value>
+  <name>metric.systemmetrics.eth0.rxpck/s</name>
+  <value>eth0_rxpcks</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.load_fifteen</name>
-  <value>LoadFifteen</value>
+  <name>metric.systemmetrics.eth0.rxdrop/s</name>
+  <value>eth0_rxdrops</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.load_five</name>
-  <value>LoadFive</value>
+  <name>metric.systemmetrics.eth0.txerr/s</name>
+  <value>eth0_txerrs</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.load_one</name>
-  <value>LoadOne</value>
+  <name>metric.systemmetrics.eth0.txbyt/s</name>
+  <value>eth0_txbyts</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.mem_buffers%</name>
-  <value>MemBuffersPercentage</value>
+  <name>metric.systemmetrics.eth0.txpck/s</name>
+  <value>eth0_txpcks</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.mem_cached%</name>
-  <value>MemCachedPercentage</value>
+  <name>metric.systemmetrics.eth0.txdrop/s</name>
+  <value>eth0_txdrops</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.mem_user%</name>
-  <value>MemUserPercentage</value>
+  <name>metric.systemmetrics.eth1.rxerr/s</name>
+  <value>eth1_rxerrs</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.mem_shared%</name>
-  <value>MemSharedPercentage</value>
+  <name>metric.systemmetrics.eth1.rxbyt/s</name>
+  <value>eth1_rxbyts</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.disk_max_busy%</name>
-  <value>MaxDiskBusyPercentage</value>
+  <name>metric.systemmetrics.eth1.rxpck/s</name>
+  <value>eth1_rxpcks</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.bytes_in</name>
-  <value>NetworkInKBps</value>
+  <name>metric.systemmetrics.eth1.rxdrop/s</name>
+  <value>eth1_rxdrops</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.bytes_out</name>
-  <value>NetworkOutKBps</value>
+  <name>metric.systemmetrics.eth1.txerr/s</name>
+  <value>eth1_txerrs</value>
 </property>
 
 <property>
-  <name>metric.disks.perdisk.node.device.sda.rKb/s</name>
-  <value>DiskAReadKBps</value>
+  <name>metric.systemmetrics.eth1.txbyt/s</name>
+  <value>eth1_txbyts</value>
 </property>
 
 <property>
-  <name>metric.disks.perdisk.node.device.sda.wKb/s</name>
-  <value>DiskAWriteKBps</value>
+  <name>metric.systemmetrics.eth1.txpck/s</name>
+  <value>eth1_txpcks</value>
 </property>
 
 <property>
-  <name>metric.disks.perdisk.node.device.sdb.rKb/s</name>
-  <value>DiskBReadKBps</value>
+  <name>metric.systemmetrics.eth1.txdrop/s</name>
+  <value>eth1_txdrops</value>
 </property>
 
 <property>
-  <name>metric.disks.perdisk.node.device.sdb.wKb/s</name>
-  <value>DiskBWriteKBps</value>
+  <name>metric.systemmetrics.sda.rkb/s</name>
+  <value>sda_rkbs</value>
 </property>
 
 <property>
-  <name>metric.disks.perdisk.node.device.sdc.rKb/s</name>
-  <value>DiskCReadKBps</value>
+  <name>metric.systemmetrics.sda.wkb/s</name>
+  <value>sda_wkbs</value>
 </property>
 
 <property>
-  <name>metric.disks.perdisk.node.device.sdc.wKb/s</name>
-  <value>DiskCWriteKBps</value>
+  <name>metric.systemmetrics.sdb.rkb/s</name>
+  <value>sdb_rkbs</value>
 </property>
 
 <property>
-  <name>metric.disks.perdisk.node.device.sdd.rKb/s</name>
-  <value>DiskDReadKBps</value>
+  <name>metric.systemmetrics.sdb.wkb/s</name>
+  <value>sdb_wkbs</value>
 </property>
 
 <property>
-  <name>metric.disks.perdisk.node.device.sdd.wKb/s</name>
-  <value>DiskDWriteKBps</value>
+  <name>metric.systemmetrics.sdc.rkb/s</name>
+  <value>sdc_rkbs</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.disk_total</name>
-  <value>DiskTotalGB</value>
+  <name>metric.systemmetrics.sdc.wkb/s</name>
+  <value>sdc_wkbs</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.disk_used%</name>
-  <value>DiskUsedPercentage</value>
+  <name>metric.systemmetrics.sdd.rkb/s</name>
+  <value>sdd_rkbs</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.pkts_in</name>
-  <value>PacketsIn</value>
+  <name>metric.systemmetrics.sdd.wkb/s</name>
+  <value>sdd_wkbs</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.pkts_out</name>
-  <value>PacketsOut</value>
+  <name>metric.systemmetrics.%idle</name>
+  <value>cpu_idle_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.swap_in_kBps</name>
-  <value>SwapInKBps</value>
+  <name>metric.systemmetrics.%nice</name>
+  <value>cpu_nice_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.pernode.node.swap_out_kBps</name>
-  <value>SwapOutKBps</value>
+  <name>metric.systemmetrics.%sys</name>
+  <value>cpu_system_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..#nodes</name>
-  <value>nodes</value>
+  <name>metric.systemmetrics.%user</name>
+  <value>cpu_user_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..#pids</name>
-  <value>pids</value>
+  <name>metric.systemmetrics.cpu_hi%</name>
+  <value>cpu_hirq_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgBlurbs</name>
-  <value>avgBlurbs</value>
+  <name>metric.systemmetrics.cpu_si%</name>
+  <value>cpu_sirq_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgBlurbRate%</name>
-  <value>avgBlurbRate</value>
+  <name>metric.systemmetrics.%iowait</name>
+  <value>iowait_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgCpuIdle%</name>
-  <value>avgCpuIdle</value>
+  <name>metric.systemmetrics.mem_buffers_pcnt</name>
+  <value>mem_buffers_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgCpuNice</name>
-  <value>avgCpuNice</value>
+  <name>metric.systemmetrics.mem_cached_pcnt</name>
+  <value>mem_cached_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgCpuNice%</name>
-  <value>avgCpuNicePercent</value>
+  <name>metric.systemmetrics.%memused</name>
+  <value>mem_used_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgCpuSystem%</name>
-  <value>avgCpuSystem</value>
+  <name>metric.systemmetrics.eth0_busy_pcnt</name>
+  <value>eth0_busy_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgCpuUser%</name>
-  <value>avgCpuUser</value>
+  <name>metric.systemmetrics.eth1_busy_pcnt</name>
+  <value>eth1_busy_pcnt</value>
 </property>
- 
+
 <property>
-  <name>metric.systemstate.percluster..avgCpuIowait%</name>
-  <value>avgCpuIowait</value>
+  <name>metric.systemmetrics.sda.%util</name>
+  <value>sda_busy_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgCpuIrq%</name>
-  <value>avgCpuIrq</value>
+  <name>metric.systemmetrics.sdb.%util</name>
+  <value>sdb_busy_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgCpuSoftirq%</name>
-  <value>avgCpuSoftirq</value>
+  <name>metric.systemmetrics.sdc.%util</name>
+  <value>sdc_busy_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgCpuBusy%</name>
-  <value>avgCpuBusy</value>
+  <name>metric.systemmetrics.sdd.%util</name>
+  <value>sdd_busy_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgLoadOne</name>
-  <value>avgLoadOne</value>
+  <name>metric.systemmetrics.swap_used_pcnt</name>
+  <value>swap_used_pcnt</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgLoadFive</name>
-  <value>avgLoadFive</value>
+  <name>report.db.primary.key.df</name>
+  <value>timestamp</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgLoadFifteen</name>
-  <value>avgLoadFifteen</value>
+  <name>metric.df.available</name>
+  <value>available</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgMemBuffers%</name>
-  <value>avgMemBuffers</value>
+  <name>metric.df.used</name>
+  <value>used</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgMemCached</name>
-  <value>avgMemCached</value>
+  <name>metric.df.use%</name>
+  <value>used_percent</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgMemCached%</name>
-  <value>avgMemCachedPercent</value>
+  <name>metric.df.mounted-on</name>
+  <value>mount</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgMemFree</name>
-  <value>avgMemFree</value>
+  <name>metric.df.filesystem</name>
+  <value>fs</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgMemFree%</name>
-  <value>avgMemFreePercent</value>
+  <name>metric.df.csource</name>
+  <value>host</value>
 </property>
 
+<!-- dfs name node metrics -->
 <property>
-  <name>metric.systemstate.percluster..avgMemUser</name>
-  <value>avgMemUser</value>
+  <name>report.db.primary.key.hadoop_dfs_namenode</name>
+  <value>timestamp</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgMemUser%</name>
-  <value>avgMemUserPercent</value>
+  <name>metric.hadoop_dfs_namenode.csource</name>
+  <value>host</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgMemShared</name>
-  <value>avgMemShared</value>
+  <name>metric.hadoop_dfs_namenode.addblockops</name>
+  <value>add_block_ops</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgMemShared%</name>
-  <value>avgMemSharedPercent</value>
+  <name>metric.hadoop_dfs_namenode.blockscorrupted</name>
+  <value>blocks_corrupted</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgMemTotal</name>
-  <value>avgMemTotal</value>
+  <name>metric.hadoop_dfs_namenode.createfileops</name>
+  <value>create_file_ops</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgSwapTotal</name>
-  <value>avgSwapTotal</value>
+  <name>metric.hadoop_dfs_namenode.deletefileops</name>
+  <value>delete_file_ops</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgSwapInKbps</name>
-  <value>avgSwapInKbps</value>
+  <name>metric.hadoop_dfs_namenode.filescreated</name>
+  <value>files_created</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgSwapOutKbps</name>
-  <value>avgSwapOutKbps</value>
+  <name>metric.hadoop_dfs_namenode.filesrenamed</name>
+  <value>files_renamed</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgBytesIn</name>
-  <value>avgBytesIn</value>
+  <name>metric.hadoop_dfs_namenode.getblocklocations</name>
+  <value>get_block_locations</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgBytesOut</name>
-  <value>avgBytesOut</value>
+  <name>metric.hadoop_dfs_namenode.getlistingops</name>
+  <value>get_listing_ops</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgPktsIn</name>
-  <value>avgPktsIn</value>
+  <name>metric.hadoop_dfs_namenode.safemodetime</name>
+  <value>safe_mode_time</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgPktsOut</name>
-  <value>avgPktsOut</value>
+  <name>metric.hadoop_dfs_namenode.syncs_avg_time</name>
+  <value>syncs_avg_time</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgDiskFree</name>
-  <value>avgDiskFree</value>
+  <name>metric.hadoop_dfs_namenode.syncs_num_ops</name>
+  <value>syncs_num_ops</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgDiskTotal</name>
-  <value>avgDiskTotal</value>
+  <name>metric.hadoop_dfs_namenode.transactions_avg_time</name>
+  <value>transactions_avg_time</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgDiskUsed%</name>
-  <value>avgDiskUsed</value>
+  <name>metric.hadoop_dfs_namenode.transactions_num_ops</name>
+  <value>transactions_num_ops</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgPartMaxUsed%</name>
-  <value>avgPartMaxUsed</value>
+  <name>metric.hadoop_dfs_namenode.blockreport_avg_time</name>
+  <value>block_report_avg_time</value>
 </property>
 
 <property>
-  <name>metric.systemstate.percluster..avgDiskMaxBusy%</name>
-  <value>avgDiskMaxBusy</value>
+  <name>metric.hadoop_dfs_namenode.blockreport_num_ops</name>
+  <value>block_report_num_ops</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..#nodes</name>
-  <value>nodes</value>
+  <name>metric.hadoop_dfs_namenode.fsimageloadtime</name>
+  <value>fs_image_load_time</value>
 </property>
 
+<!-- dfs data node -->
 <property>
-  <name>metric.dfs.hdfs throughput..#pids</name>
-  <value>pids</value>
+  <name>report.db.primary.key.hadoop_dfs_datanode</name>
+  <value>timestamp</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..bytesWrittenPerSec</name>
-  <value>bytesWrittenPerSec</value>
+  <name>metric.hadoop_dfs_datanode.hostname</name>
+  <value>host</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..blocksRemovedPerSec</name>
-  <value>blocksRemovedPerSec</value>
+  <name>metric.hadoop_dfs_datanode.blockreports_avg_time</name>
+  <value>block_reports_avg_time</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..readsFromRemoteClientPerSec</name>
-  <value>readsFromLocalRemotePerSec</value>
+  <name>metric.hadoop_dfs_datanode.blockreports_num_ops</name>
+  <value>block_reports_num_ops</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..writesFromLocalClientPerSec</name>
-  <value>writesFromLocalClientPerSec</value>
+  <name>metric.hadoop_dfs_datanode.block_verification_failures</name>
+  <value>block_verification_failures</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..blocksVerifiedPerSec</name>
-  <value>blocksVerifiedPerSec</value>
+  <name>metric.hadoop_dfs_datanode.blocks_read</name>
+  <value>blocks_read</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..blocksWrittenPerSec</name>
-  <value>blocksWrittenPerSec</value>
+  <name>metric.hadoop_dfs_datanode.blocks_removed</name>
+  <value>blocks_removed</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..blockVerificationFailuresPerSec</name>
-  <value>blockVerificationFailuresPerSec</value>
+  <name>metric.hadoop_dfs_datanode.blocks_replicated</name>
+  <value>blocks_replicated</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..#restarts</name>
-  <value>restarts</value>
+  <name>metric.hadoop_dfs_datanode.blocks_verified</name>
+  <value>blocks_verified</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..blocksReplicatedPerSec</name>
-  <value>blocksReplicatedPerSec</value>
+  <name>metric.hadoop_dfs_datanode.blocks_written</name>
+  <value>blocks_written</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..bytesReadPerSec</name>
-  <value>bytesReadPerSec</value>
+  <name>metric.hadoop_dfs_datanode.bytes_read</name>
+  <value>bytes_read</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..writesFromRemoteClientPerSec</name>
-  <value>writesFromLocalRemotePerSec</value>
+  <name>metric.hadoop_dfs_datanode.bytes_written</name>
+  <value>bytes_written</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..readsFromLocalClientPerSec</name>
-  <value>readsFromLocalClientPerSec</value>
+  <name>metric.hadoop_dfs_datanode.copyblockop_avg_time</name>
+  <value>copy_block_op_avg_time</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..blocksReadPerSec</name>
-  <value>blocksReadPerSec</value>
+  <name>metric.hadoop_dfs_datanode.copyblockop_num_ops</name>
+  <value>copy_block_op_num_ops</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..readBlockOperationsPerSec</name>
-  <value>readBlockOperationsPerSec</value>
+  <name>metric.hadoop_dfs_datanode.heartbeats_avg_time</name>
+  <value>heart_beats_avg_time</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..numberReadBlockOperations</name>
-  <value>numberReadBlockOperations</value>
+  <name>metric.hadoop_dfs_datanode.heartbeats_num_ops</name>
+  <value>heart_beats_num_ops</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..writeBlockOperationsPerSec</name>
-  <value>writeBlockOperationsPerSec</value>
+  <name>metric.hadoop_dfs_datanode.readblockop_avg_time</name>
+  <value>read_block_op_avg_time</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..numberWriteBlockOperations</name>
-  <value>numberWriteBlockOperations</value>
+  <name>metric.hadoop_dfs_datanode.readblockop_num_ops</name>
+  <value>read_block_op_num_ops</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..readMetadataOperationsPerSec</name>
-  <value>readMetadataOperationsPerSec</value>
+  <name>metric.hadoop_dfs_datanode.readmetadataop_avg_time</name>
+  <value>read_metadata_op_avg_time</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..numberReadMetadataOperations</name>
-  <value>numberReadMetadataOperations</value>
+  <name>metric.hadoop_dfs_datanode.readmetadataop_num_ops</name>
+  <value>read_metadata_op_num_ops</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..copyBlockOperationsPerSec</name>
-  <value>copyBlockOperationsPerSec</value>
+  <name>metric.hadoop_dfs_datanode.reads_from_local_client</name>
+  <value>reads_from_local_client</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..numberCopyBlockOperations</name>
-  <value>numberCopyBlockOperations</value>
+  <name>metric.hadoop_dfs_datanode.reads_from_remote_client</name>
+  <value>reads_from_remote_client</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..replaceBlockOperationsPerSec</name>
-  <value>replaceBlockOperationsPerSec</value>
+  <name>metric.hadoop_dfs_datanode.replaceblockop_avg_time</name>
+  <value>replace_block_op_avg_time</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..numberReplaceBlockOperations</name>
-  <value>numberReplaceBlockOperations</value>
+  <name>metric.hadoop_dfs_datanode.replaceblockop_num_ops</name>
+  <value>replace_block_op_num_ops</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..heartBeatsPerSec</name>
-  <value>heartBeatsPerSec</value>
+  <name>metric.hadoop_dfs_datanode.writeblockop_avg_time</name>
+  <value>write_block_op_avg_time</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..numberHeartBeats</name>
-  <value>numberHeartBeats</value>
+  <name>metric.hadoop_dfs_datanode.writeblockop_num_ops</name>
+  <value>write_block_op_num_ops</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..blockReportsPerSec</name>
-  <value>blockReportsPerSec</value>
+  <name>metric.hadoop_dfs_datanode.writes_from_local_client</name>
+  <value>writes_from_local_client</value>
 </property>
 
 <property>
-  <name>metric.dfs.hdfs throughput..numberBlockReports</name>
-  <value>numberBlockReports</value>
+  <name>metric.hadoop_dfs_datanode.writes_from_remote_client</name>
+  <value>writes_from_remote_client</value>
 </property>
 
+<!-- dfs fs name system status -->
 <property>
-  <name>metric.dfs.individual datanode throughput.node.#pids</name>
-  <value>pids</value>
+  <name>report.db.primary.key.hadoop_dfs_fsnamesystem</name>
+  <value>timestamp</value>
 </property>
-
 <property>
-  <name>metric.dfs.individual datanode throughput.node.bytesWrittenPerSec</name>
-  <value>bytesWrittenPerSec</value>
+  <name>metric.hadoop_dfs_fsnamesystem.csource</name>
+  <value>host</value>
 </property>
-
 <property>
-  <name>metric.dfs.individual datanode throughput.node.blocksRemovedPerSec</name>
-  <value>blocksRemovedPerSec</value>
+  <name>metric.hadoop_dfs_fsnamesystem.blockstotal</name>
+  <value>blocks_total</value>
 </property>
-
 <property>
-  <name>metric.dfs.individual datanode throughput.node.readsFromRemoteClientPerSec</name>
-  <value>readsFromLocalRemotePerSec</value>
+  <name>metric.hadoop_dfs_fsnamesystem.capacityremaininggb</name>
+  <value>capacity_remaining_gb</value>
 </property>
-
 <property>
-  <name>metric.dfs.individual datanode throughput.node.writesFromLocalClientPerSec</name>
-  <value>writesFromLocalClientPerSec</value>
+  <name>metric.hadoop_dfs_fsnamesystem.capacitytotalgb</name>
+  <value>capacity_total_gb</value>
 </property>
-
 <property>
-  <name>metric.dfs.individual datanode throughput.node.blocksVerifiedPerSec</name>
-  <value>blocksVerifiedPerSec</value>
+  <name>metric.hadoop_dfs_fsnamesystem.capacityusedgb</name>
+  <value>capacity_used_gb</value>
 </property>
-
 <property>
-  <name>metric.dfs.individual datanode throughput.node.blocksWrittenPerSec</name>
-  <value>blocksWrittenPerSec</value>
+  <name>metric.hadoop_dfs_fsnamesystem.filestotal</name>
+  <value>files_total</value>
 </property>
-
 <property>
-  <name>metric.dfs.individual datanode throughput.node.blockVerificationFailuresPerSec</name>
-  <value>blockVerificationFailuresPerSec</value>
+  <name>metric.hadoop_dfs_fsnamesystem.pendingreplicationblocks</name>
+  <value>pending_replication_blocks</value>
 </property>
-
 <property>
-  <name>metric.dfs.individual datanode throughput.node</name>
-  <value>node</value>
+  <name>metric.hadoop_dfs_fsnamesystem.scheduledreplicationblocks</name>
+  <value>scheduled_replication_blocks</value>
 </property>
-
 <property>
-  <name>metric.dfs.individual datanode throughput.node.#restarts</name>
-  <value>restarts</value>
+  <name>metric.hadoop_dfs_fsnamesystem.totalload</name>
+  <value>total_load</value>
 </property>
-
 <property>
-  <name>metric.dfs.individual datanode throughput.node.blocksReplicatedPerSec</name>
-  <value>blocksReplicatedPerSec</value>
+  <name>metric.hadoop_dfs_fsnamesystem.underreplicatedblocks</name>
+  <value>under_replicated_blocks</value>
 </property>
 
+<!-- dfs fsdirectory metrics -->
 <property>
-  <name>metric.dfs.individual datanode throughput.node.bytesReadPerSec</name>
-  <value>bytesReadPerSec</value>
+  <name>report.db.primary.key.hadoop_dfs_fsdirectory</name>
+  <value>timestamp</value>
 </property>
 
 <property>
-  <name>metric.dfs.individual datanode throughput.node.writesFromRemoteClientPerSec</name>
-  <value>writesFromLocalRemotePerSec</value>
+  <name>metric.hadoop_dfs_fsdirectory.csource</name>
+  <value>host</value>
+</property>
+ <property>
+  <name>metric.hadoop_dfs_fsdirectory.files_deleted</name>
+  <value>files_deleted</value>
 </property>
 
+<!-- hadoop jvm metrics -->
 <property>
-  <name>metric.dfs.individual datanode throughput.node.readsFromLocalClientPerSec</name>
-  <value>readsFromLocalClientPerSec</value>
+  <name>report.db.primary.key.hadoop_jvm_metrics</name>
+  <value>timestamp</value>
 </property>
 
 <property>
-  <name>metric.dfs.individual datanode throughput.node.blocksReadPerSec</name>
-  <value>blocksReadPerSec</value>
+  <name>metric.hadoop_jvm_metrics.csource</name>
+  <value>host</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node</name>
-  <value>node</value>
+  <name>metric.hadoop_jvm_metrics.gctimemillis</name>
+  <value>gc_timemillis</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.#pids</name>
-  <value>pids</value>
+  <name>metric.hadoop_jvm_metrics.gccount</name>
+  <value>gc_count</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.#restarts</name>
-  <value>restarts</value>
+  <name>metric.hadoop_jvm_metrics.logerror</name>
+  <value>log_error</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.filesDeletedPerSec</name>
-  <value>filesDeletedPerSec</value>
+  <name>metric.hadoop_jvm_metrics.logfatal</name>
+  <value>log_fatal</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.filesCreatedPerSec</name>
-  <value>filesCreatedPerSec</value>
+  <name>metric.hadoop_jvm_metrics.loginfo</name>
+  <value>log_info</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.filesOpenedPerSec</name>
-  <value>filesOpenedPerSec</value>
+  <name>metric.hadoop_jvm_metrics.logwarn</name>
+  <value>log_warn</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.filesRenamedPerSec</name>
-  <value>filesRenamedPerSec</value>
+  <name>metric.hadoop_jvm_metrics.memheapcommittedm</name>
+  <value>mem_heap_committed_m</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.filesListedPerSec</name>
-  <value>filesListedPerSec</value>
+  <name>metric.hadoop_jvm_metrics.memheapusedm</name>
+  <value>mem_heap_used_m</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.numberOfTransactionsPerSec</name>
-  <value>numberOfTransactionsPerSec</value>
+  <name>metric.hadoop_jvm_metrics.memnonheapcommittedm</name>
+  <value>mem_non_heap_committed_m</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.AverageTransactions</name>
-  <value>AverageTransactions</value>
+  <name>metric.hadoop_jvm_metrics.memnonheapusedm</name>
+  <value>mem_non_heap_used_m</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.timeInSafeMode</name>
-  <value>timeInSafeMode</value>
+  <name>metric.hadoop_jvm_metrics.processname</name>
+  <value>process_name</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.numberOfSyncsPerSec</name>
-  <value>numberOfSyncsPerSec</value>
+  <name>metric.hadoop_jvm_metrics.threadsblocked</name>
+  <value>threads_blocked</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.AverageSyncTime</name>
-  <value>AverageSyncTime</value>
+  <name>metric.hadoop_jvm_metrics.threadsnew</name>
+  <value>threads_new</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.fsImageLoadTime</name>
-  <value>fsImageLoadTime</value>
+  <name>metric.hadoop_jvm_metrics.threadsrunnable</name>
+  <value>threads_runnable</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.BlocksCorrupted</name>
-  <value>BlocksCorrupted</value>
+  <name>metric.hadoop_jvm_metrics.threadsterminated</name>
+  <value>threads_terminated</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.numberOfBlockReportPerSec</name>
-  <value>numberOfBlockReportPerSec</value>
+  <name>metric.hadoop_jvm_metrics.threadstimedwaiting</name>
+  <value>threads_timed_waiting</value>
 </property>
 
 <property>
-  <name>metric.dfs.namenode operations.node.AverageBlockReportTime</name>
-  <value>AverageBlockReportTime</value>
+  <name>metric.hadoop_jvm_metrics.threadswaiting</name>
+  <value>threads_waiting</value>
 </property>
 
+<!-- hadoop map/reduce metrics -->
 <property>
-  <name>metric.dfs.fsnamesystem status.node</name>
-  <value>node</value>
+  <name>report.db.primary.key.hadoop_mapred_jobtracker</name>
+  <value>timestamp</value>
 </property>
 
 <property>
-  <name>metric.dfs.fsnamesystem status.node.FilesTotal</name>
-  <value>FilesTotal</value>
+  <name>metric.hadoop_mapred_jobtracker.csource</name>
+  <value>host</value>
 </property>
-
 <property>
-  <name>metric.dfs.fsnamesystem status.node.BlocksTotal</name>
-  <value>BlocksTotal</value>
+  <name>metric.hadoop_mapred_jobtracker.jobs_completed</name>
+  <value>jobs_completed</value>
+</property>
+<property>
+  <name>metric.hadoop_mapred_jobtracker.jobs_submitted</name>
+  <value>jobs_submitted</value>
 </property>
-
 <property>
-  <name>metric.dfs.fsnamesystem status.node.CapacityTotalGB</name>
-  <value>CapacityTotalGB</value>
+  <name>metric.hadoop_mapred_jobtracker.maps_completed</name>
+  <value>maps_completed</value>
 </property>
 
 <property>
-  <name>metric.dfs.fsnamesystem status.node.CapacityUsedGB</name>
-  <value>CapacityUsedGB</value>
+  <name>metric.hadoop_mapred_jobtracker.maps_launched</name>
+  <value>maps_launched</value>
 </property>
 
 <property>
-  <name>metric.dfs.fsnamesystem status.node.CapacityRemainingGB</name>
-  <value>CapacityRemainingGB</value>
+  <name>metric.hadoop_mapred_jobtracker.reduces_completed</name>
+  <value>reduces_completed</value>
 </property>
 
 <property>
-  <name>metric.dfs.fsnamesystem status.node.TotalLoad</name>
-  <value></value>
+  <name>metric.hadoop_mapred_jobtracker.reduces_launched</name>
+  <value>reduces_launched</value>
 </property>
 
+<!-- hadoop rpc metrics -->
 <property>
-  <name>metric.dfs.fsnamesystem status.node.PendingReplicationBlocks</name>
-  <value>PendingReplicationBlocks</value>
+  <name>report.db.primary.key.hadoop_rpc_metrics</name>
+  <value>timestamp</value>
 </property>
 
 <property>
-  <name>metric.dfs.fsnamesystem status.node.UnderReplicatedBlocks</name>
-  <value>UnderReplicatedBlocks</value>
+  <name>metric.hadoop_rpc_metrics.csource</name>
+  <value>host</value>
 </property>
 
 <property>
-  <name>metric.dfs.fsnamesystem status.node.ScheduledReplicationBlocks</name>
-  <value>ScheduledReplicationBlocks</value>
+  <name>metric.hadoop_rpc_metrics.rpcprocessingtime_avg_time</name>
+  <value>rpc_processing_time_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.rpcprocessingtime_num_ops</name>
+  <value>rpc_processing_time_num_ops</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..#nodes</name>
-  <value>nodes</value>
+  <name>metric.hadoop_rpc_metrics.getbuildversion_avg_time</name>
+  <value>get_build_version_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getbuildversion_num_ops</name>
+  <value>get_build_version_num_ops</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getjobcounters_avg_time</name>
+  <value>get_job_counters_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getjobcounters_num_ops</name>
+  <value>get_job_counters_num_ops</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getjobprofile_avg_time</name>
+  <value>get_job_profile_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getjobprofile_num_ops</name>
+  <value>get_job_profile_num_ops</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getjobstatus_avg_time</name>
+  <value>get_job_status_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getjobstatus_num_ops</name>
+  <value>get_job_status_num_ops</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getnewjobid_avg_time</name>
+  <value>get_new_job_id_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getnewjobid_num_ops</name>
+  <value>get_new_job_id_num_ops</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getprotocolversion_avg_time</name>
+  <value>get_protocol_version_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getprotocolversion_num_ops</name>
+  <value>get_protocol_version_num_ops</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getsystemdir_avg_time</name>
+  <value>get_system_dir_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.getsystemdir_num_ops</name>
+  <value>get_system_dir_num_ops</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.gettaskcompletionevents_avg_time</name>
+  <value>get_task_completion_events_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.gettaskcompletionevents_num_ops</name>
+  <value>get_task_completion_events_num_ops</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.gettaskdiagnostics_avg_time</name>
+  <value>get_task_diagnostics_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.gettaskdiagnostics_num_ops</name>
+  <value>get_task_diagnostics_num_ops</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.heartbeat_avg_time</name>
+  <value>heartbeat_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.heartbeat_num_ops</name>
+  <value>heartbeat_num_ops</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.submitjob_avg_time</name>
+  <value>submit_job_avg_time</value>
+</property>
+<property>
+  <name>metric.hadoop_rpc_metrics.submitjob_num_ops</name>
+  <value>submit_job_num_ops</value>
 </property>
 
+<!-- Hod Machine -->
 <property>
-  <name>metric.rpcmetrics.rpc metrics..AverageRpcQueueTime_num_ops</name>
-  <value>AverageRpcQueueTime_num_ops</value>
+  <name>metric.hodmachine.machine</name>
+  <value>host</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..AverageRpcQueueTime_avg_time</name>
-  <value>AverageRpcQueueTime_avg_time</value>
+  <name>metric.hodmachine.hodid</name>
+  <value>hodid</value>
 </property>
 
+<!-- Hod Job -->
 <property>
-  <name>metric.rpcmetrics.rpc metrics..AverageRpcQueueTime_num_ops</name>
-  <value>AverageRpcQueueTime_num_ops</value>
+  <name>metric.hodjob.hodid</name>
+  <value>HodID</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..AverageRpcProcessingTime_num_ops</name>
-  <value>AverageRpcProcessingTime_num_ops</value>
+  <name>metric.hodjob.userid</name>
+  <value>UserID</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..AverageRpcProcessingTime_avg_time</name>
-  <value>AverageRpcProcessingTime_avg_time</value>
+  <name>metric.hodjob.status</name>
+  <value>Status</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..RpcDiscarded_num_ops</name>
-  <value>RpcDiscarded_num_ops</value>
+  <name>metric.hodjob.timequeued</name>
+  <value>TimeQueued</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..RpcDiscarded_avg_time</name>
-  <value>RpcDiscarded_avg_time</value>
+  <name>metric.hodjob.starttime</name>
+  <value>StartTime</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..register_num_ops</name>
-  <value>register_num_ops</value>
+  <name>metric.hodjob.endtime</name>
+  <value>EndTime</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..register_avg_time</name>
-  <value>register_avg_time</value>
+  <name>metric.hodjob.numofmachines</name>
+  <value>NumOfMachines</value>
 </property>
 
+<!-- MSSRGraph -->
 <property>
-  <name>metric.rpcmetrics.rpc metrics..getProtocolVersion_num_ops</name>
-  <value>getProtocolVersion_num_ops</value>
+  <name>report.db.primary.key.mssrgraph</name>
+  <value>timestamp</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..getProtocolVersion_avg_time</name>
-  <value>getProtocolVersion_avg_time</value>
+  <name>metric.mssrgraph.jobid</name>
+  <value>job_id</value>
+</property>
+<property>
+  <name>metric.mssrgraph.count</name>
+  <value>count</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..sendHeartbeat_num_ops</name>
-  <value>sendHeartbeat_num_ops</value>
+  <name>metric.mssrgraph.type</name>
+  <value>type</value>
 </property>
 
+<!-- Map Reduce Job Counters -->
 <property>
-  <name>metric.rpcmetrics.rpc metrics..sendHeartbeat_avg_time</name>
-  <value>sendHeartbeat_avg_time</value>
+  <name>report.db.primary.key.mrjobcounters</name>
+  <value>timestamp</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..blockReport_num_ops</name>
-  <value>blockReport_num_ops</value>
+  <name>metric.mrjobcounters.file_systems_hdfs_bytes_read</name>
+  <value>hdfs_bytes_read</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..blockReport_avg_time</name>
-  <value>blockReport_avg_time</value>
+  <name>metric.mrjobcounters.file_systems_hdfs_bytes_written</name>
+  <value>hdfs_bytes_written</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.file_systems_local_bytes_read</name>
+  <value>local_bytes_read</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.file_systems_local_bytes_written</name>
+  <value>local_bytes_written</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.job_counters__data-local_map_tasks</name>
+  <value>data_local_map_tasks</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.job_counters__launched_map_tasks</name>
+  <value>launched_map_tasks</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.job_counters__launched_reduce_tasks</name>
+  <value>launched_reduce_tasks</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.jobid</name>
+  <value>job_id</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.map-reduce_framework_combine_input_records</name>
+  <value>combine_input_records</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.map-reduce_framework_combine_output_records</name>
+  <value>combine_output_records</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.map-reduce_framework_map_input_bytes</name>
+  <value>map_input_bytes</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.map-reduce_framework_map_output_bytes</name>
+  <value>map_output_bytes</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.map-reduce_framework_map_input_records</name>
+  <value>map_input_records</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.map-reduce_framework_map_output_records</name>
+  <value>map_output_records</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.map-reduce_framework_reduce_input_groups</name>
+  <value>reduce_input_groups</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.map-reduce_framework_reduce_input_records</name>
+  <value>reduce_input_records</value>
+</property>
+<property>
+  <name>metric.mrjobcounters.map-reduce_framework_reduce_output_records</name>
+  <value>reduce_output_records</value>
 </property>
 
+<!-- Database summarization intervals -->
 <property>
-  <name>metric.rpcmetrics.rpc metrics..getBlockLocations_num_ops</name>
-  <value>getBlockLocations_num_ops</value>
+  <name>consolidator.table.dfs_namenode</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..heartbeat_num_ops</name>
-  <value>heartbeat_num_ops</value>
+  <name>consolidator.table.dfs_datanode</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..versionRequest_num_ops</name>
-  <value>versionRequest_num_ops</value>
+  <name>consolidator.table.hadoop_rpc</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..setPermission_num_ops</name>
-  <value>setPermission_num_ops</value>
+  <name>consolidator.table.cluster_hadoop_rpc</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>metric.rpcmetrics.rpc metrics..rollFsImage_num_ops</name>
-  <value>rollFsImage_num_ops</value>
+  <name>consolidator.table.hadoop_mapred</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>conversion.metric.systemstate.pernode.node.bytes_in</name>
-  <value>0.001</value>
+  <name>consolidator.table.hadoop_jvm</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>conversion.metric.systemstate.pernode.node.bytes_out</name>
-  <value>0.001</value>
+  <name>consolidator.table.system_metrics</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>util.perfMetrics</name>
-  <value>CPUBusy,FreeMemPercentage,MaxDiskBusyPercentage,NetworkInKBps,NetworkOutKBps,DiskAReadKBps,DiskBReadKBps,DiskCReadKBps,DiskDReadKBps,DiskAWriteKBps,DiskBWriteKBps,DiskCWriteKBps,DiskDWriteKBps,DiskUsedPercentage</value>
+  <name>consolidator.table.dfs_throughput</name>
+  <value>5,30,180,720</value>
 </property>
 
-<!-- Database summarization intervals -->
 <property>
-  <name>consolidator.table.dfs_namenode</name>
-  <value>5,30,120</value>
+  <name>consolidator.table.node_activity</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>consolidator.table.SimonData</name>
-  <value>5,30,120</value>
+  <name>consolidator.table.dfs_fsnamesystem</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>consolidator.table.rpc_metrics</name>
-  <value>5,30,120</value>
+  <name>consolidator.table.disk</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>consolidator.table.dfs_throughput</name>
-  <value>5,30,120</value>
+  <name>consolidator.table.cluster_disk</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>consolidator.table.dfs_individual</name>
-  <value>5,30,120</value>
+  <name>consolidator.table.cluster_system_metrics</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>consolidator.table.cluster_systemstate</name>
-  <value>5,30,120</value>
+  <name>consolidator.table.hod_job_digest</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>consolidator.table.NodeActivity</name>
-  <value>5,30,120</value>
+  <name>consolidator.table.hod_machine</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>consolidator.table.HodJobDigest</name>
-  <value>5,30,120</value>
+  <name>consolidator.table.HodJob</name>
+  <value>5,30,180,720</value>
 </property>
 
 <property>
-  <name>consolidator.table.dfs_fsnamesystem_status</name>
-  <value>5,30,120</value>
+  <name>consolidator.table.user_util</name>
+  <value>5,30,180,720</value>
 </property>
+
 </configuration>
+

+ 2 - 2
src/contrib/chukwa/conf/system-data-loader.properties

@@ -1,6 +1,6 @@
-log4j.rootLogger=INFO, stdout, R
+log4j.rootLogger=INFO, R
 log4j.appender.R=org.apache.hadoop.chukwa.inputtools.log4j.ChukwaDailyRollingFileAppender
-log4j.appender.R.File=${CHUKWA_HOME}/logs/${RECORD_TYPE}.log
+log4j.appender.R.File=${CHUKWA_LOG_DIR}/${RECORD_TYPE}.log
 log4j.appender.R.recordType=${RECORD_TYPE}
 log4j.appender.R.layout=org.apache.log4j.PatternLayout
 log4j.appender.R.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n

+ 10 - 0
src/contrib/chukwa/docs/README

@@ -78,7 +78,17 @@ on localhost:
   - Edit bin/chukwa-config.sh to match your system configuration
   
   - In the chukwa root directory, say ``bash bin/processSinkFiles.sh'' 
+
+*  Configuring and starting Database
+
+  - Download MySQL connector from http://dev.mysql.com/downloads/connector/j/5.1.html
+
+  - Copy mysql-connector-*.jar to Chukwa lib directory.
+
+  - Configure mysql, and run:  mysql -u root dbname < database_create_table to install new database schema from Chukwa conf directory.
   
+  - Configure JDBC driver name in conf/chukwa-env.sh: com.mysql.jdbc.Driver
+
 *  Configuring and starting HICC
 
   - Download Apache Tomcat from http://tomcat.apache.org/download-60.cgi

+ 84 - 79
src/contrib/chukwa/docs/paper/chukwa_08.tex

@@ -86,68 +86,90 @@ A well documented open source toolset for handling monitoring data thus solves a
 
 We did not aim to solve the problem of real-time monitoring for failure detection, which systems such as Ganglia already do well. Rather, we wanted a system that would process large volumes of data, in a timescale of minutes, not seconds, to detect more subtle conditions, and to aid in failure diagnosis. Human engineers do not generally react on a timescale of seconds, and so a processing delay of a few minutes is not a concern for us.
 
-Such a data collection system ought not be developed in isolation. Many problems inherent in monitoring can be solved by relying on existing scalable distributed computing platforms. In particular, we believe that a distributed filesystem is a natural tool for data storage, and that MapReduce is a natural way of processing monitoring data. Leveraging this existing infrastructure drastically reduces the engineering difficulty inherent in large-scale monitoring.
+%In particular, we believe that a distributed filesystem is a natural tool for data storage, and that MapReduce is a natural way of processing monitoring data. Leveraging this existing infrastructure drastically reduces the engineering difficulty inherent in large-scale monitoring.
+
+%One of our overriding considerations was to use existing components as much as possible.  In particular, we sought to leverage the engineering that has gone into Hadoop's distributed filesystem 
  
  We are in the process of building a system, which we call Chukwa, to demonstrate that practical large-scale can be readily built atop this existing infrastructure. \footnote{In Hindu mythology, Chukwa is the turtle that holds up Maha-pudma, the elephant that hold up the world.  This name is especially appropriate for us, since the the Hadoop mascot is a yellow elephant.}   
  it uses Hadoop's distributed file system (HDFS) as its data store, and relies on MapReduce jobs to process the data. By leveraging these existing tools, Chukwa can scale to thousands of nodes in both collection and analysis capacities, while providing a standardized and familiar framework  for processing the collected data. Many components of Chukwa are pluggable, allowing easy customization and enhancement.
  
-The core components of Chukwa are largely complete, and we expect the system to enter production use at Yahoo! within the next few months. We have some initial operational experience, and preliminary performance metrics.    We begin by discussing our goals and requirements in some detail.  We then describe our design, explaining our motivation for various decisions. We then present some performance data, and conclude by offering some comparisons with related work.
+The core components of Chukwa are largely complete, and we expect the system to enter production use at Yahoo! within the next few months. We have some initial operational experience, and preliminary performance metrics.    We begin by discussing our goals and requirements in some detail.  We then describe our design, explaining our motivation for various decisions. We next present some performance data, and conclude by offering some comparisons with related work.
 
-\section{Goals and requirements} 
+\section{Motivation and requirements} 
 
+We intend to use Chukwa to monitor multiple clusters of several thousand hosts, potentially generating several terabytes of data per day. Our goals in designing Chukwa were based on survey of our cluster user's functional requirements and performance demands.
 
-There are two key figures of merit for monitoring systems such as Chukwa: scale and latency.  We intend to use Chukwa to monitor multiple clusters of several thousand hosts, potentially generating several terabytes of data per day.  Our initial goal was a latency of less than ten minutes between the generation of data on cluster nodes and its availability in HDFS for centralized analysis using Hadoop MapReduce. We believe that our design can operate effectively with latencies closer to a minute, as discussed in our conclusion.  Few useful actions can be taken on timescales significantly shorter than that, and therefore we do not believe this limit will be problematic for us.%this statement may need more support
+We expect Chukwa to be used by four different (though overlapping) constituencies:  Hadoop users, cluster operators, cluster managers, and Hadoop developers.  These different groups have different functional requirements:
 
-The initial goal for Chukwa is to monitor and analyze several-thousand-node Hadoop deployments. Nothing in our design restricts us from monitoring other clusters, however our goals are primarily driven by the motivating example of Hadoop.  Analysis encompasses many goals:  
 \begin{itemize}
-\item
-Providing canned reports to meet known needs for accounting, capacity planning, performance characterization, utilization.  Some examples:
-%\begin{itemize}
-%\item 
-%Summarize grid usage: disk usage / CPU total, grouped by account, project, and user.  
-%\item 
-%  Measure performance improvement across Hadoop versions
-%  \item 
-%  Assess the quality of service based on waiting time, node availability, and service uptime
-%\end{itemize}
-These can be done in a straightfoward way via MapReduce jobs, provided that the data has been stored suitably.  
-\item
- Reducing the number and extent of outages through trend analysis and anomaly detection to identify at-risk elements such as compute nodes, name node, switches 
-\item
-  Contributing to Hadoop development in robustness and performance by making possible the analysis of jobs and infrastructure along many different dimensions
-\item
-   Reducing the number of false alerts and increasing the value and confidence level of true alerts through correlation of metrics and environmental information
-\item
-    Reducing the time and effort required to identify and resolve cluster issues by highlighting trouble spots and by correlating metrics and environmental information to provide the required context and history, as well as by bringing together all required tools in one place.
+\item \textbf{Hadoop Users} will ask how far along their jobs are, and what resources are available for future jobs. They need access to the logs and output from their jobs.  
+
+\item \textbf{Operators} need to be notified of hardware failures and performance anomalies. They need to be warned about resource shortages, such as storage exhaustion.  
+
+\item \textbf{Managers} need guidance in provisioning, and in apportioning costs. This means that they need tools for analyzing past usage by users and groups, and for projecting future demands.   They need access to figures of merit, such as average job waiting time.
+
+\item \textbf{Hadoop Developers} need information about the performance in operation, bottlenecks within Hadoop, common failure patterns, and so forth.
 \end{itemize}
 
-Chukwa can collect a large and open-ended set of time series metrics and logs, as well as  slowly changing dimensions such as machine configuration, related to the state, behavior, characterization, and performance of a Hadoop cluster and processes running on it. Stored data will be available indefinitely, limited only by storage capacity, and not by retrieval speed.  
+Fortunately these different demands boil down to a comparatively small set of technical requirements.  Chukwa must collect a large and open-ended set of time series metrics and logs, as well as  slowly changing dimensions such as machine configuration.  Stored data should be available promptly, and should remain available indefinitely. Efficient querying and analysis of large data volumes is essential.
 
-%Chukwa should be robust against agent crashes
+Our initial goal was to be able to monitor Hadoop clusters of 2000 nodes, outputting 5 to 6 MB of data per second, and to have collected data available for processing within ten minutes. Few operational Hadoop clusters today are larger than 2000 nodes, and thus that figure represents a reasonable initial operating capability.  In section 4 of this paper, we report the operational measurements that justify our target data rate.
 
-\section{Data Collection}
+While having all data available immediately after collection might be desirable, it is not actually crucial. Systems such as Nagios or Ganglia work well for real-time monitoring of metrics such as CPU load.   Human administrators can take few useful actions on timescales shorter than a few minutes, and so low-latency execution of more complex processing is not a priority.
 
-\begin{figure}
- \includegraphics[width=75mm]{chukwa0.jpg}
-\caption{The Chukwa collection architecture} 
-\end{figure}
 
+\section{Architecture}
+
+\begin{figure*}
+ \includegraphics[width=150mm]{chukwa_pipeline.png}
+\caption{The Chukwa Pipeline, showing how long data is retained at each stage.} 
+\end{figure*}
+
+\iffalse
 \begin{figure}
   \includegraphics[width=75mm]{chukwa1.jpg}
 \caption{The agent side}
 \end{figure}
+\fi
 
-To achieve our scalability goals, we opted early in the design process to make HDFS our chief storage technology. HDFS installations can store petabytes of data, and support fairly high throughput. HDFS also facilitates parallel processing of stored data with MapReduce.
 
-Chukwa thus breaks into two halves -- the pipeline for getting data into a convenient format in HDFS, and a toolkit for analyzing data using MapReduce and displaying it in a helpful way. A pipelined approach to data collection aids flexibility, by providing natural points for adding functionality. We discuss each section of the pipeline in turn, before discussing the user interface and analysis tools offered by Chukwa.
+At the heart of any data collection system is a pipeline to pump data from where it is generated to where it is stored. The requirements at the endpoints dictate the design of the system in the middle. To meet its goals, Chukwa needs flexible, dynamically controllable data sources, and a high performance, large scale storage system. %Generated data ought to be available for processing shortly after generation
+ It also needs a suitable framework for analyzing the large volumes of collected data.
 
 \subsection{Adaptors}
 
-The particular data being collected about a machine will change with time, and from machine to machine. As Hadoop tasks start and stop, different log files must be monitored. We might want to increase our collection rate if we detect anomalies.  And of course, the monitoring package on a given machine must be customized according to the services running on that machine: it makes no sense to collect Hadoop metrics on an NFS server.
+Data sources need to be dynamically controllable because the particular data being collected from a machine changes over time, and varies from machine to machine. For example, as Hadoop tasks start and stop, different log files must be monitored. We might want to increase our collection rate if we detect anomalies.  And of course, it makes no sense to collect Hadoop metrics on an NFS server. 
+
+These dynamically controllable data sources are known in Chukwa as \textit{adaptors}, since they generally are wrapping some other data source, such as a file or a Unix command-line tool.  At present, Chukwa includes adaptors to collect Hadoop logs, application metrics, and system telemetry. We expect to write adaptors for tasks like counting recoverable disk read errors, retrieving causal logs from X-Trace \cite{xtrace}, and monitoring operating system and Java virtual machine state.
+
+%\subsection{Adaptors}
+% 
+%As a result, we bundle data collection into small dynamically loadable Adaptors which run within a local agent process on each machine. This process is left permanently running, and is restarted automatically if it crashes. The agent process is responsible for starting and stopping adaptors in response to external commands.  It also provides two crucial services to adaptors. First, it is responsible for forwarding chunks over HTTP to the collectors, where they are written to stable storage. Second, it is responsible for making regular checkpoints of adaptor state, and restarting adaptors at the appropriate position after a crash.  This checkpoint mechanism ensures that data is appropriately resent after a crash.
+
+
+\subsection{Storage}
+
+The scalability challenges in large-scale monitoring systems primarily concern the data storage and analysis components, since that is where data from multiple machines is brought together. We determined from the outset to rely on Hadoop's HDFS as our storage component. Hadoop HDFS installations can store petabytes of data, and support high throughput; 20 MB/sec for one writer is typical in operational deployments, with total cluster throughput routinely in excess of a gigabyte per second. HDFS also facilitates parallel processing of stored data with MapReduce.
+
+Unfortunately, HDFS is not designed for the sort of workloads associated with monitoring. HDFS aims to handle large files and high write rates from comparatively small numbers of writers. It is not designed for thousands of concurrent low-rate writers, and millions of small files. Worse, writes to a file are not visible to readers until the file is closed, and stable versions of HDFS do not allow closed files to be reopened for writing. As a result, some care must be taken in using HDFS to support continuous rather than batch processing.
+ Much of the Chukwa design was driven by the need to reconcile our many sporadic data sources with HDFS's performance characteristics and semantics.
+ 
+\subsection{Collectors and Agents}
+
+Chukwa resolves these conflicting demands by adding additional pipeline stages between the adaptors and the HDFS data store: \textit{collectors} and \textit{agents}.
+
+Rather than have each adaptor write directly to HDFS, data is sent across the network to a \textit{collector} process, that does the HDFS writes.  Each collector receives data from several hundred hosts, and writes all this data to a single \textit{sink file}, consisting of chunks of data plus metadata describing each chunk's source and format. Periodically, collectors close their sink files, rename them to mark them available for processing, and resume writing a new file.  Data is sent to collectors over HTTP, since this allows us to write our collector as a Java servlet. This in turn lets us use standard Java servlet containers for connection management. This is in keeping with the Chukwa philosophy of leveraging existing infrastructure when possible. 
+
+Collectors thus drastically reduce the number of HDFS files generated by Chukwa, from one per machine or adaptor per unit time, to a handful per cluster.  The decision to put collectors between data sources and the data store has other benefits. Collectors hide the details of the HDFS file system in use, such as its Hadoop version, from the adaptors.  This is a significant aid to configuration.  It is especially helpful when using Chukwa to monitor a development cluster running a different version of Hadoop or when using Chukwa to monitor a non-Hadoop cluster.  
+
+The second of our intermediate stages, agents, are less fundamental to the design. They exist primarily to provide various services to adaptors, and thus to make adaptors easier to write. Agents are long-running processes on each machine being monitored by Chukwa.  Each agent process is restarted automatically if it crashes. The agent provides three chief services to adaptors. First, the agent is responsible for starting and stopping adaptors in response to external commands.
+Second, it is responsible for forwarding chunks over HTTP to the collectors, where they are written to stable storage. 
+Third, it is responsible for making regular checkpoints of adaptor state, and restarting adaptors at the appropriate position after a crash.  
+
 
-As a result, we bundle data collection into small dynamically loadable Adaptors.  At present, Chukwa includes adaptors to collect Hadoop logs, application metrics, and system telemetry. We expect to write adaptors for tasks like counting recoverable disk read errors, retrieving causal logs from X-Trace \cite{xtrace}, and monitoring operating system and Java virtual machine state. 
 
 %FIXME: ruby/failmon
+
 %The output of an adaptor consists of chunks, each containing one or more semantically meaningful records.  These records, such as lines from a log file or batches of metrics, must be kept together.  This means that adaptors must parse files intensively enough to detect record boundaries, and they must correctly handle buffering.
 
 %These costs are offset by important advantages. Keeping records intact allows collectors to run filters or triggers against data streams, without having to buffer partial lines. In the presence of load balancing across collectors, these filters will still behave properly. Perhaps most importantly, this allows the map phase of the periodic MapReduce process to extract metadata from records and use this metadata as a sort key: a large performance win.  If records were split across collectors, this metadata could be inaccessible.
@@ -155,25 +177,18 @@ As a result, we bundle data collection into small dynamically loadable Adaptors.
 %FIXME: say something here
 %After a crash, the local agent restarts each adaptor, passing sequence number of the last byte of their output that was successfully sent to a collector.  Adaptors that read from files can seek to that offset before starting.  Adaptors that output ephemeral data, such as CPU load, can simply begin numbering output from this point, to mask  any crash-induced discontinuity from downstream users. 
 
-\subsection{Agent}
-
-Adaptors run within a local agent process on each machine. This process is left permanently running, and is restarted automatically if it crashes. The agent process is responsible for starting and stopping adaptors in response to external commands.  It also provides two crucial services to adaptors. First, it is responsible for forwarding chunks over HTTP to the collectors, where they are written to stable storage. Second, it is responsible for making regular checkpoints of adaptor state, and restarting adaptors at the appropriate position after a crash.  This checkpoint mechanism ensures that data is appropriately resent after a crash.
-
-We opted to put most of the recovery logic in the agents, rather than the collectors. This removes state from the collectors. We considered having agents check with collectors to find out what the last thing they sent was. However, each agent would have to check with each collector, resulting in an exorbitant number of connections if many agents fail and reboot simultaneously.
+%\subsection{Agent}
 
-\subsection{Collector}
+%Adaptors run within a local agent process on each machine. This checkpoint mechanism ensures that data is appropriately resent after a crash.
 
-The collector is a web service (implemented as a Java servlet) that receives chunks via HTTP, and writes them to stable storage in the Hadoop distributed file system.  
+%We opted to put most of the recovery logic in the agents, rather than the collectors. This removes state from the collectors. We considered having agents check with collectors to find out what the last thing they sent was. However, each agent would have to check with each collector, resulting in an exorbitant number of connections if many agents fail and reboot simultaneously.
 
-Each collector writes to a sink files in the Hadoop distributed file system. Each sink file contains a sequence of chunks, potentially from many different streams.  Every few minutes, collectors ``rotate'' sink files, by closing their current sink files, and marking them as available for processing. While it would make sense to use HDFS files as circular buffers, the filesystem semantics forbid this.  Instead, each collector closes its current sink file every few minutes, and marks it available for processing.  
-
-An alternate design would be to have each local agent dump its results directly into HDFS. Unfortunately, HDFS does not perform well for filesystems with large numbers of files, due to contention and space limits at the Namenode.  The many-to-one relationship between hosts and collectors helps significantly, although if Chukwa deployments got big enough, some more drastic step might be needed.
-
-Another advantage of having collectors is that it allows us to hide the details of the Chukwa storage cluster, such as its Hadoop version from the agents.  This is important, for instance, when using Chukwa to monitor a development cluster running an unstable version of Hadoop or when using Chukwa to monitor a non-Hadoop cluster.  
 
 \subsection{Demux and archiving}
 
-A pair of MapReduce jobs run every few minutes, taking these sink files as input.  The first job simply archives all the collected data, without processing or interpreting it.  The second job parses out structured data from some of the logs, and loads this structured data into a data store.   For performance reasons, we do record parsing in the Map phase, and extract key-value pairs. Example records include job history reports, task durations, and so forth. We then use the shuffle and sort phases of the Reduce to organize records semantically by application timestamp and type. This forced us to design our adaptors to have adaptors output meaningful records, and not arbitrary chunks of streams. %% FIXME: recast last sentence
+A pair of MapReduce jobs runs every few minutes, taking all the available sink files as input.  The first job simply archives all the collected data, without processing or interpreting it.  The second job parses out structured data from some of the logs, and loads this structured data into a data store.  
+
+% For performance reasons, we do record parsing in the Map phase, and extract key-value pairs. Example records include job history reports, task durations, and so forth. We then use the shuffle and sort phases of the Reduce to organize records semantically by application timestamp and type. This forced us to design our adaptors to have adaptors output meaningful records, and not arbitrary chunks of streams. %% FIXME: recast last sentence
 
  These datastores are also pluggable. For now, we use HDFS files, one file per cluster, per data type, and time period.  So for instance there would be one file for all of a particular clusters datanode logs, for the period from noon to 1pm on a given day.   This is only an interim solution, and we are evaluating various more suitable data stores, with support for structured queries.  Hive, an HDFS-backed data warehouse might also be a good fit here. \cite{hive} 
  % Hive supports a subset of SQL for queries, which is particularly appealing for the context of ad-hoc analytics. 
@@ -183,11 +198,8 @@ A pair of MapReduce jobs run every few minutes, taking these sink files as input
 
 Data stored in HDFS in a structured format can be processed straightforwardly with MapReduce jobs. We envision a library of ``canned'' MapReduce jobs for tasks like finding common failure modes, correlating events in the logs with slowdowns, discovering flakey machines, and so forth.  Since Chukwa data is split into different files based on content, these jobs take as input only a small fraction of the total data volume, and therefore can run relatively quickly.  Most structured storage systems, including Hive and Hypertable, include their own query interfaces. We expect that these interfaces will be used by users who want to do simple ad-hoc queries over stored Chukwa data, with MapReduce being reserved for more complex processing.
 
-
 \section{Data Analysis and Display}
 
-
-
 Collected data is only as useful as the analysis that can be done on it.  To ease analysis of collected data, we've built a flexible, configurable, ``portal-style'' web interface to Chukwa, termed the Hadoop Infrastructure Care Center (HICC). A configurable interface is not simply a frill --- it is necessary, since different users have very different data analysis needs.
 
 \begin{figure}
@@ -197,49 +209,31 @@ Collected data is only as useful as the analysis that can be done on it.  To eas
 \caption{HICC displaying some DataNode metrics}
 \end{figure}
 
-\subsection{The Use Model}
-
-We expect HICC to be used by four different (though overlapping) constituencies:   users, operators, managers, and developers.  These different groups have different characteristic questions:
-
-
-\begin{itemize}
-\item Users will ask how far along their jobs are, and what resources are available for future jobs. They need access to the logs and output from their jobs.  
-
-\item Operators need to be notified of hardware failures and performance anomalies. They need to be warned about resource shortages, such as storage exhaustion.  
-
-\item Managers need guidance in provisioning, and in apportioning costs. This means that they need tools for analyzing past usage by users and groups, and for projecting future demands.   They need access to figures of merit, such as average job waiting time.
-
-\item Developers need information about the performance in operation, bottlenecks within Hadoop, common failure patterns, and so forth.
-
-\end{itemize}
-
-\subsection{How HICC works}
 
 In practice, a single individual often fulfills more than one of these roles, or some portion of a role. As a result, there is a compelling need to allow individuals to mix and match different components. We chose to do this by bundling each query, or family of queries, into a widget. HICC users can assemble their HICC workspace by selecting widgets from a catalog, in exactly the way that they can customize their personal Yahoo! or Google portal pages.
 
 Some of these components will display the results of canned map-reduce jobs run against data in  Chukwa storage.  Others will perform on-the-fly queries against SQL databases.    Still others might display telemetry collected with Ganglia, or report on recently opened failure tickets.  
 
-HICC stores several kinds of widget state.  There is a global context, through which different widgets can ``collaborate'' and share data. There is a dashboard view, which stores information about the user's preferred layout, and display settings.  There is a widget descriptor file, for storing widget configuration and parameters.   
+% HICC stores several kinds of widget state.  There is a global context, through which different widgets can ``collaborate'' and share data. There is a dashboard view, which stores information about the user's preferred layout, and display settings.  There is a widget descriptor file, for storing widget configuration and parameters.   
 
 %HICC is not intended for exploratory, ad-hoc queries. For that, we expect to rely on the query interface of our structured data store.  Since this store is not yet available, we have been using a MySQL database.
 
+
 \section{Evaluation}
 
 %Chukwa is currently in development, and we have not yet been able to do large-scale tests.  However, we have reason to be confident that Chukwa will comfortably meet our performance goals.
 
 Using logs from a production cluster at Yahoo!, we found that a 2000-node production cluster would generate around 5.5 MB of data per second.  Of this, the vast bulk (more than 95\%) was task tracker logs.  Metrics data accounted for more than half the remainder, with Namenode, HDFS datanode, and JobTracker logs accounting for the rest. This data rate is small enough that Chukwa should impose only very modest overhead on datacenter networks. 
 
-  We conducted a number of small experiments to verify that Chukwa could handle this sort of load.   All tests were run on an internal development cluster at Yahoo.  Machines had four 2.8 GHz Xeon processors, four IDE disks, and 3 GB of RAM, and ran Linux, with a 2.6.9 kernel.  %kernel version 2.6.9-55
-
-We checked for performance limitations at three places: the collector, the HDFS cluster and the map-reduce job.
-To assess collector throughput, we disabled HDFS writes, and measured the throughput between a single agent and collector, connected by gigabit ethernet.  We found that throughput was around 80 MB/sec, and that this limit was imposed by network latency.  Handling this input took only half of one CPU on the collector side, which is an affordable overhead. 
+We conducted a number of small experiments to verify that Chukwa could handle this load.   All tests were run on an internal development cluster at Yahoo.  Machines had four 2.8 GHz Xeon processors, four IDE disks, and 3 GB of RAM, and ran Linux, with a 2.6.9 kernel.  %kernel version 2.6.9-55
+There are two potential bottlenecks in Chukwa that we evaluated in detail, the collector, and the map-reduce job.  At present, collector throughput is more than adequate, and the demux job is the limiting phase in processing.
 
-A single Chukwa collector was easily able to saturate the HDFS back-end with very modest CPU cost. Our five-node HDFS test cluster saturated at about 25 megabytes per second.  Even writing 25 megabytes per second, the collector was only using about 10\% of a single core. This is an encouraging number, since it suggests that even small monitoring clusters have ample write bandwidth for current as well as future data collection demands.  
+To measure collector performance, we ran Chukwa on a 400 node test cluster. We configured nodes in this cluster to report data at many times the normal operational rate, emulating a much larger cluster.  In this configuration, the test cluster generated 14.4 megabytes of monitoring data per second.  A single collector was able to keep up with this data volume, and write it to HDFS; in a 30 minute test run, machine utilization never rose much above 50\%.  At this rate, we are bumping into the single-writer throughput limits imposed by HDFS, rather than any Chukwa-specific limits. Higher Chukwa bandwidth could be achieved by simply adding more writers.
 
-%The bottleneck in our current design is the demux job.  As mentioned, limitations in Hadoop restrict performance 
-%we expect that our write bandwidth would increase if we added additional collectors, since using more than one writer enables the HDFS datanodes to concurrently write across multiple physical disks.  
+%The scalability limits we observed in more intensive experiments were caused by thread limits in Jetty, the Servlet container we are using at present. These limits can likely be overcome by more careful configuration. However, 7.2 MB/sec significantly exceeds both our performance goals and the rate at which we can process incoming data, so we have yet to perform this optimization. 
 
-As mentioned, the bottleneck in our design is the Demux job. We found that our Demux job can easily keep up with incoming data.  Using four worker nodes, we were able to process a gigabyte of data in two and a half minutes; for a throughput of 6.6 MB/sec, which exceeds our target data rate. Larger data volumes improved efficiency significantly:  with 2 GB of data, the processing rate went up to 8.5 MB/sec. Optimizing MapReduce jobs is a reasonably well understood science, and we believe that significant gains can be readily achieved here. 
+At present, the rate-limiting phase of Chukwa is the Demux job. Using five worker nodes, our MapReduce job can process two gigabytes of metrics data in around three and a half minutes. We conducted five trials on the same 2 GB of test data.  Completion times ranged from 3:25 minutes to 3:34, with a mean of 3:30.  This means that we can can process six minutes' of incoming data in three and a half minutes, thus keeping up with the incoming data flow and achieving our ten minute target latency.  
+Optimizing MapReduce jobs is fairly routine engineering at this point, and we believe that significant gains can be achieved here. 
 
 These results show that Chukwa can maintain latencies well under our ten minute target, while imposing very modest overheads on the cluster: five Chukwa nodes are only 0.25\% of our notional 2000-node cluster. We expect to be able to maintain these latency targets as we scale up the number of nodes being monitored.  Ramping up the size of MapReduce jobs is routine, and the engineering issues are well understood. Even for monitoring hundreds of thousands of nodes, Chukwa's data volumes would be significantly smaller than those seen in our production web indexing clusters.
 
@@ -249,14 +243,21 @@ These results show that Chukwa can maintain latencies well under our ten minute
 
 Chukwa represents a design point in between two existing classes of systems: log collection frameworks on the one hand, and network management systems on the other.  Chukwa intends to combine the abundance of data display tools of existing NMS systems, with the high throughput and robustness expected of log collection frameworks.
 
-The syslog protocol supported streaming logs across the network as long ago as the late 1980s.  However, syslog had serious defects: no clear solution to the discovery, load balancing, or failure handing problems.  The Scribe system apparently solves some of these problems, but unfortunately, no details of scribe have been published.
+The syslog protocol supported streaming logs across the network as long ago as the late 1980s.  However, syslog had serious defects: no clear solution to the discovery, load balancing, or failure handing problems.  Facebook's Scribe \cite{scribe} system apparently solves some of these problems, but unfortunately, no details of Scribe have been published.
 
 Chukwa has some similarity with network monitoring systems such as Nagios, Ganglia, or Tivoli Monitoring \cite{Ganglia, Nagios, tivmon}. The three systems differ in emphasis, but have important commonalities.  All are capable of collecting and storing substantial volumes of metrics data. All include tools for displaying this data.  Nagios and Tivoli monitoring have centralized architectures, while Ganglia is decentralized.  Ganglia, unfortunately, is heavily adapted towards numeric time-series data, and provides minimal support for the sort of complex text-processing necessary for our applications.
 
+Chukwa, however, differs in crucial respects from these current systems. Today's monitoring systems are focused primarily on collection, with storage being a secondary priority.  Chukwa is designed for far higher data rates; metrics data, which is essentially all that Ganglia and Nagios are used to collect, is only a few percent of the data we will capture in operational settings. 
+
+With hundreds of gigabytes of data being collected per day, processing the stored data becomes a key bottleneck.  Chukwa's design was optimized precisely for storage and batch processing of collected data.  While MapReduce is routinely used at these scales, no currently available monitoring system makes provision for large-scale data intensive processing. 
+
+
 \section{Conclusion}
 %\label{sec:conclusion}
 
-Chukwa provides a flexible and powerful toolkit for analyzing displaying collected data.  We expect Chukwa to be very useful in monitoring and managing large system deployments.
+Chukwa demonstrates that a high performance distributed monitoring system can readily be built atop existing distributed data collection frameworks. The Hadoop distributed file system supports petabytes of stored data and hundreds of megabytes per second of write throughput, enough for even very demanding monitoring applications. MapReduce provides a suitable framework for organizing and analyzing these data volumes.
+
+% Chukwa provides flexible and powerful tools for analyzing and displaying collected monitoring data.  We expect Chukwa to be very useful in monitoring and managing large system deployments.
 
 Building Chukwa on top of Hadoop resulted in a few design quirks, and a modest latency penalty.  However, it greatly simplified implementation, and leverages the substantial amount of work going into Hadoop.  Hadoop 0.19, which will be released within a few months, should significantly improve the performance of short-running Map tasks, which will allow us to efficiently operate Chukwa on short timescales.
 
@@ -285,6 +286,10 @@ Joydeep Sen Sarma.  ``Hive as a contrib project''
 \bibitem{xtrace}
 Rodrigo Fonseca, George Porter, Randy H. Katz, Scott Shenker, and Ion Stoica.  X-Trace: A Pervasive Network Tracing Framework.
 \newblock In \textit{4th USENIX Symposium on Networked Systems Design \& Implementation (NSDI'07)}, Cambridge, MA, USA, April 2007.
+\bibitem{scribe}
+Scribe logfile aggregation system described by Facebook's Jeff Hammerbacher
+\url{https://issues.apache.org/jira/browse/HADOOP-2206?focusedCommentId=12542775#action_12542775}
+
 \end{thebibliography}
 
 \end{document}

+ 0 - 23
src/contrib/chukwa/hadoop-packaging/build-tarball.sh

@@ -1,23 +0,0 @@
-#!/bin/bash
-
-
-#leave all jars in build
-HADOOP_DIR=hadoop-0.18.0-mac01
-TARFILE=hadoop18.tar
-rm -r hadoop-0.18.0-mac01
-mkdir $HADOOP_DIR
-mkdir $HADOOP_DIR/bin $HADOOP_DIR/conf $HADOOP_DIR/lib $HADOOP_DIR/chukwa
-cp ../build/chukwa-hadoop-0.0.1-client.jar $HADOOP_DIR/lib
-cp log4j.properties.templ $HADOOP_DIR/conf/log4j.properties
-tar xf $TARFILE $HADOOP_DIR/bin/hadoop
-patch $HADOOP_DIR/bin/hadoop < patchhadoop.patch
-svn export ../bin $HADOOP_DIR/chukwa/bin
-cp ../bin/VERSION $HADOOP_DIR/chukwa/bin
-svn export ../conf $HADOOP_DIR/chukwa/conf
-svn export ../lib $HADOOP_DIR/chukwa/lib
-cp ../hadoopjars/hadoop-0.18.0-core.jar $HADOOP_DIR/chukwa/lib
-cp ../build/*.jar $HADOOP_DIR/chukwa
-mkdir $HADOOP_DIR/chukwa/var; mkdir $HADOOP_DIR/chukwa/var/run
-cp new-chukwa-conf/* $HADOOP_DIR/chukwa/conf
-# Do something with chukwa-conf 
-tar uvf $TARFILE $HADOOP_DIR

+ 0 - 3
src/contrib/chukwa/hadoop-packaging/patchhadoop.patch

@@ -1,3 +0,0 @@
-46a47,48
-> CHUKWA_HOME=`dirname "$0"`/../chukwa ;  CHUKWA_HOME=`cd "$CHUKWA_HOME"; pwd` ;  export CHUKWA_HOME
-> nohup bash $CHUKWA_HOME/bin/agent.sh &> /tmp/chukwa_out < /dev/null &

+ 201 - 0
src/contrib/chukwa/lib/jstl-LICENSE.txt

@@ -0,0 +1,201 @@
+ Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

+ 344 - 0
src/contrib/chukwa/lib/jstl-README.txt

@@ -0,0 +1,344 @@
+---------------------------------------------------------------------------
+Standard Tag Library 1.1 -- SOURCE DISTRIBUTION
+---------------------------------------------------------------------------
+Thanks for downloading the source code of the Standard tag library, 
+an implementation of the JavaServer Pages(tm)(JSP) 
+Standard Tag Library (JSTL).
+
+JSTL is an effort of the Java Community Process (JCP) and
+comes out of the JSR-052 expert group. For more information on JSTL,
+please go to http://java.sun.com/products/jstl.
+
+---------------------------------------------------------------------------
+BUILD ENVIRONMENT SETUP
+
+---
+J2SE (Java 2 Standard Edition)
+
+Download and install Sun's distribution of J2SE 1.4.2 (or higher) 
+for your operating system platform. J2SE can be found at 
+http://java.sun.com/j2se.
+
+Sun's distribution of J2SE 1.4.2 includes many of the libraries that
+standard 1.1 depends on. However, please note that standard 1.1 will
+compile and run on previous versions of J2SE (from 1.3 up to 1.4.1) as
+long as the standard 1.1 dependencies not included in these versions
+of the J2SE are properly setup.  See section 'LIBRARY DEPENDENCIES'
+for details.
+
+  - Set a JAVA_HOME environment variable to point at the directory 
+    where J2SE is installed.
+  - Add the JAVA_HOME/bin directory to your PATH.
+
+---
+Ant
+
+Download and install version 1.5 or higher of the Jakarta Ant Project
+distribution. Ant can be fount at http://ant.apache.org.
+
+  - Set the ANT_HOME environment variable to point at your Ant 
+    distribution directory 
+  - Add the ANT_HOME/bin directory to your PATH.
+
+---------------------------------------------------------------------------
+LIBRARY DEPENDENCIES
+
+This version of the Standard Tag Library has the following compile-time
+dependencies:
+
+   1. Dependencies not included in J2SE:
+      - Servlet 2.4
+      - JSP 2.0
+
+   2. Dependencies included in newer J2SEs (1.4.2 and higher)
+      - JAXP 1.2 
+      - Xalan 2.5 
+      - JDBC Standard Extension 2.0
+
+Since all of the dependencies in (2) are included in Sun's
+distribution of J2SE 1.4.2 (or higher), this is therefore the J2SE
+version of choice to compile and run the standard tag library.
+
+If you still need the jar files for the dependencies listed in (2), 
+instructions on where you can get these jar files are included below.
+
+---
+build.properties
+
+- Copy the file standard/build_sample_standard.properties to build.properties.
+
+- Edit build.properties and make the following modifications:
+    - Set the "base.dir" property in build.properties to the base directory
+      of your 'standard' distribution. It must be an absolute path.
+    - Set the jar file properties to the absolute path and filename 
+      for the jar files required to build the standard tag library
+      (see below).
+
+---
+Servlet 2.4 and JSP 2.0
+
+Download and install the Servlet 2.4 and JSP 2.0 APIs.  The jar files
+for these APIs may be found in distributions of jakarta-servlet-api-5-bin
+and jakarta-jsp-api-5-bin at http://cvs.apache.org/builds/jakarta-tomcat-5/nightly.  
+They are also included in Tomcat 5, available at http://jakarta.apache.org/tomcat.
+
+Set the following properties in build.properties to the
+file paths of the jars:
+  - servlet24.jar
+  - jsp20.jar
+
+---
+JAXP 1.2
+[required only if building with versions of J2SE that do not include
+the JAXP APIs, such as J2SE 1.3]
+
+The JAXP 1.2 jar files can be obtained in the Java Web Services
+Developer Pack (JWSDP) available at 
+http://java.sun.com/products/jwsdp.
+
+Set the following properties in build.properties to the
+file paths of the jars:
+  - jaxp-api.jar
+  - dom.jar
+  - sax.jar
+  - xercesImpl.jar
+
+---
+Xalan 2.5
+[required only if building with J2SE 1.3 up to J2SE 1.4.1]
+
+The Xalan jar file can be obtained in the Java Web Services
+Developer Pack (JWSDP) available at 
+http://java.sun.com/products/jwsdp, as well as from 
+Apache at http://xml.apache.org/xalan-j.
+
+Set the "xalan.jar" property in build.properties to the
+jar file of Xalan.
+
+If using jdk 1.3, put xalan.jar in the lib directory
+of ant so XSLT transformations of documentation can be 
+properly done.
+
+---
+JDBC Standard Extension 2.0
+[required only if building with J2SE 1.3]
+
+The JDBC 2.0 Optional Package can be obtained from:
+http://java.sun.com/products/jdbc/download.html
+
+Set the "jdbc2_0-stdext.jar" property in build.properties to the
+JDBC 2.0 Standard Extensions jar file path.
+
+---------------------------------------------------------------------------
+Building the Standard tag library
+
+To build the distribution set your current directory to the 'standard' 
+directory into which you unpacked the distribution.
+
+Build 'standard' by executing ant in a shell. Some common build targets
+include:
+
+       > ant         <-- builds the intermediate form of the library,
+                         documentation, and example targets
+       > ant dist    <-- builds all the distribution targets
+       > ant clean   <-- deletes intermediate results so that target can
+                         be rebuilt from scratch.
+
+Two directory hierarchies are created to contain the results of the
+build:
+
+{base.dir}/
+    build/           <-- Contains intermediate form results of
+                         building standard custom library
+    dist/            <-- Contains the files that will be included
+                         in the binary distribution of the
+                         standard project
+
+The following directory and files are created when doing a build:
+
+   * build/standard - Location of all directories and files built for the 
+     standard taglib.
+   * build/standard/standard - Results of the build process
+     (classes, jar files, tlds)
+   * build/standard/standard-doc/ - Files used to create the
+     standard-doc.war file
+   * build/standard/standard-examples/ - Files used to create the 
+     standard-examples.war file.
+
+The following directory and files are created when doing a distribution
+build:
+
+   * dist/standard/ - Location of all files built for a binary
+     distribution of the taglib.
+   * dist/standard/README - Information to use the binary distribution
+     of the standard tablib.
+   * dist/standard/javadoc/ - The javadocs
+   * dist/standard/lib/ - The standard jar files: jstl.jar and
+     standard.jar
+   * dist/standard/tld/ - Directory with the Tag Lib Descriptors for 
+     the tag library.
+   * dist/standard/standard-doc.war - Tag Library documentation
+     war file.
+   * dist/standard/standard-examples.war - Tag Library examples
+     war file.
+   * dist/standard/tld - Directory with the Tag Lib Descriptors for 
+     the tag library.
+
+---------------------------------------------------------------------------
+USING THE STANDARD TAG LIBRARY
+
+See the README file of the binary distribution you have built with these
+instructions.
+
+---------------------------------------------------------------------------
+
+UNIT TESTING THE STANDARD TAG LIBRARY
+
+---------------------------------------------------------------------------
+OVERVIEW OF THE UNIT TEST FRAMEWORK
+
+The Unit test directory hierarchy should mimic the RI implementation hierarchy. This way, you have a mapping between the unit test and the file that it covers.
+
+If there's no appropriate mapping you can put the test in the general test directory:
+
+<JSTL_HOME>/test/org/apache/taglibs/standard 
+
+The Test should be written using Junit Test name conventions and start with
+"Test*.java"
+
+Here is the Unit test directory hierarchy:
+
+  <JSTL_HOME>/
+    test/
+      conf/                            <-- unit test configuration files
+      web/                             <-- unit test web application files
+        WEB-INF/
+      org/
+        apache/
+          taglibs/
+            standard/
+              extra/
+                spath/
+              functions/
+              lang/
+                jstl/
+              resources/
+              tag/
+                common/
+                  core/
+                  fmt/
+                  sql/
+                  xml/
+                el/
+                  core/
+                  fmt/
+                  sql/
+                  xml/
+                rt/
+                  core/
+                  fmt/
+                  sql/
+                  xml/
+              tei/
+              tlv/
+
+
+---------------------------------------------------------------------------
+UNIT TEST BUILD ENVIRONMENT SETUP
+
+In order to build the Standard Unit Test Framework, you will need to set the
+following properties and download the corresponding implementation files. See the "build_sample_standard.properties" for the definitions:
+
+        build.dir                     Base directory into which we are building
+                                      the components.
+        build.classes                 Base JSTL classes directory 
+
+        jstl.jar                      JSTL API jar file
+        standard.jar                  JSTL implementation jar file
+
+        servlet24.jar                 Servlet jar
+        jsp20.jar                     JSP jar
+
+        junit.jar                     JUnit jar
+
+        cactus.jar                    Cactus jar 
+        cactus.ant.jar                Cactus custom Ant tasks jar
+        aspectjrt.jar                 Cactus AspectJ Language Support jar
+        httpclient.jar                Cactus httpclient jar
+        commons-logging.jar           Cactus logging jar
+
+        tomcat.home                   Home directory where Tomcat 
+                                      is installed
+        tomcat.webapps                Tomcat webapps directory
+        username                      Tomcat username with manager role
+                                      privileges
+        password                      password for the user with manager
+                                      privileges
+
+        web.server.host               hostname for the running webserver
+        web.server.port               port for the running webserver
+
+        compile.debug                 debug mode for compilation
+        compile.deprecation           deprecation mode for compilation
+        compile.optimize              optimization mode for compilation
+
+---------------------------------------------------------------------------
+BUILDING THE UNIT TESTS
+
+Some common unit test build targets include:
+    > ant test                       <-- Build, deploy and run all the tests
+    > ant run.cactus                 <-- Build, deploy and run the Cactus tests
+    > ant run.junit                  <-- Build, deploy and run the Junit tests
+
+The following directories and files are created when building the unit tests:
+
+To execute the build-tests.xml unit test targets directly you can do the following:
+
+    > ant -f build-tests.xml <target>
+
+Some examples of targets available in the build-tests.xml file include:
+
+    > ant -f build-tests.xml props    <-- Print out test environment properties
+    > ant -f build-tests.xml undeploy <-- Manually undeploy the Cactus 
+                                          Tests web application
+    > ant -f build-tests.xml deploy   <-- Manually deploy the Cactus Test 
+                                          web application
+
+The following directories will be created when building the Unit Tests:
+    build/
+      standard/
+        standard/
+          test/
+            WEB-INF/
+              classes/
+                ...
+              lib/
+            org/
+                ...                  
+
+A Unit Test web application war file will also be created. It is located here:
+    build/standard/standard/jstl_test.war
+
+---------------------------------------------------------------------------
+UNIT TEST RUNTIME SETUP
+
+You will need to have the Tomcat manager administration application enabled in
+order to run the tests. The Unit Test build file uses the Tomcat manager to
+automatically deploy and undeploy the Cactus tests.
+
+The following runtime properties need to be properly set in order to successfully run the unit tests:
+
+        tomcat.webapps                Tomcat webapps directory
+        username                      Tomcat username with manager role
+                                      privileges
+        password                      password for the user with manager
+                                      privileges
+
+        web.server.host               hostname for the running webserver
+        web.server.port               port for the running webserver
+
+The Tomcat manager application may sometimes get into an unstable state after
+many web application deployments and undeployments. If the Cactus tests start
+failing for unknown reasons, stop your web container and clean out your work
+directory before restarting.
+---------------------------------------------------------------------------

+ 2 - 2
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/Chunk.java

@@ -93,13 +93,13 @@ public interface Chunk {
   public int getSerializedSizeEstimate();
   
 /**
- * @return tags from this chunk.
+ * @return name of cluster that this chunk comes from.
  * 
  */
   public String getTags();  
   
   /**
-   * Set tags to this chunk.
+   * Set the name of the cluster that this chunk comes from.
    * 
    */
     public void setTags(String tags);  

+ 3 - 3
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/ChunkImpl.java

@@ -30,7 +30,7 @@ import org.apache.hadoop.chukwa.datacollection.agent.ChukwaAgent;
 public class ChunkImpl implements org.apache.hadoop.io.Writable, Chunk 
 {
   public static int PROTOCOL_VERSION=1;
-	
+  
   private String source = "";
   private String application = "";
   private String dataType = "";
@@ -38,10 +38,10 @@ public class ChunkImpl implements org.apache.hadoop.io.Writable, Chunk
   private byte[] data = null;
   private int[] recordEndOffsets;
   private int protocolVersion=1;
-  private long seqID;
-  
   private String debuggingInfo="";
+  
   private transient Adaptor initiator;
+  long seqID;
   
   ChunkImpl() {
     this.tags = ChukwaAgent.getTags();

+ 25 - 9
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/conf/ChukwaConfiguration.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.chukwa.conf;
 
+import java.io.File;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
@@ -32,16 +34,30 @@ public class ChukwaConfiguration extends Configuration {
 	public ChukwaConfiguration(boolean loadDefaults) {
 		super();
 		if (loadDefaults) {
-		  String chukwaHome = System.getenv("CHUKWA_HOME");
-		  if (chukwaHome == null)
-		    chukwaHome = ".";
-		  log.info("chukwaHome is " + chukwaHome);
-		  
-			super.addResource(new Path(chukwaHome + "/conf/chukwa-collector-conf.xml"));
-			log.debug("added chukwa-collector-conf.xml to ChukwaConfiguration");
+
+			String chukwaHome = System.getenv("CHUKWA_HOME");
+			if (chukwaHome == null){
+				chukwaHome = ".";
+			}
+
+			if(!chukwaHome.endsWith("/"))
+			{  chukwaHome = chukwaHome + File.separator; }	
+			String chukwaConf = System.getenv("CHUKWA_CONF_DIR");
+			if (chukwaConf == null)
+			{  chukwaConf = chukwaHome + "conf" + File.separator; }
+
+			log.info("chukwaConf is " + chukwaConf);
+
+		  super.addResource(new Path(chukwaConf + "/chukwa-collector-conf.xml"));
+		  log.debug("added chukwa-collector-conf.xml to ChukwaConfiguration");
+
+		  super.addResource(new Path(chukwaConf + "/chukwa-agent-conf.xml"));
+		  log.debug("added chukwa-agent-conf.xml to ChukwaConfiguration");
+
+		  super.addResource(new Path(chukwaConf + "/hadoop-site.xml"));
+		  log.debug("added hadoop-site.xml to ChukwaConfiguration");
+
 			
-			super.addResource(new Path(chukwaHome + "/conf/chukwa-agent-conf.xml"));
-			log.debug("added chukwa-agent-conf.xml to ChukwaConfiguration");
 		}
 	}
 

+ 204 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/Aggregator.java

@@ -0,0 +1,204 @@
+/*
+ * 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.database;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.chukwa.util.DatabaseWriter;
+import org.apache.hadoop.chukwa.util.PidFile;
+
+public class Aggregator {
+	private static DatabaseConfig dbc = null;
+
+	private static Log log = LogFactory.getLog(Consolidator.class);
+	private long current = 0;
+    private static PidFile loader=null;
+
+	public Aggregator() {
+
+		dbc = new DatabaseConfig();
+		Calendar now = Calendar.getInstance();
+		current = now.getTimeInMillis();
+	}
+
+	public HashMap<String,String> findMacros(String query) {
+		boolean add=false;
+		HashMap<String,String> macroList = new HashMap<String,String>();
+		String macro="";
+	    for(int i=0;i<query.length();i++) {
+	    	if(query.charAt(i)==']') {
+	    		add=false;
+	    		if(!macroList.containsKey(macro)) {
+		    		String subString = computeMacro(macro);
+		    		macroList.put(macro,subString);	    			
+	    		}
+	    		macro="";
+	    	}
+	    	if(add) {
+	    		macro=macro+query.charAt(i);
+	    	}
+	    	if(query.charAt(i)=='[') {
+	    		add=true;
+	    	}
+	    }
+	    return macroList;
+	}
+
+	public String computeMacro(String macro) {
+		if(macro.indexOf("avg(")==0) {
+			String meta="";
+			String[] table = dbc.findTableName(macro.substring(4,macro.indexOf(")")), current, current);
+			try {
+				String cluster = System.getProperty("CLUSTER");
+				if(cluster==null) {
+					cluster="unknown";
+				}
+				DatabaseWriter db = new DatabaseWriter(cluster);
+
+			    String query = "select * from "+table[0]+" order by timestamp desc limit 1";
+	            log.debug("Query: "+query);
+	            ResultSet rs = db.query(query);
+	            if(rs==null) {
+	          	    throw new SQLException("Table is undefined.");
+	            }
+	            ResultSetMetaData rmeta = rs.getMetaData();
+	            if(rs.next()) {
+	            	boolean first=true;
+	                for(int i=1;i<=rmeta.getColumnCount();i++) {
+	                	if(!first) {
+	                		meta=meta+",";
+	                	}
+		                if(rmeta.getColumnType(i)==java.sql.Types.VARCHAR) {
+		                	meta=meta+"count("+rmeta.getColumnName(i)+") as "+rmeta.getColumnName(i);
+		                	first=false;
+		                } else if(rmeta.getColumnType(i)==java.sql.Types.DOUBLE || 
+		                		  rmeta.getColumnType(i)==java.sql.Types.INTEGER || 
+		                		  rmeta.getColumnType(i)==java.sql.Types.FLOAT) {
+		                	meta=meta+"avg("+rmeta.getColumnName(i)+")";
+		                	first=false;
+		                } else if(rmeta.getColumnType(i)==java.sql.Types.TIMESTAMP) {
+		                	// Skip the column
+		                } else {
+		                	meta=meta+"avg("+rmeta.getColumnName(i)+")";
+		                	first=false;		                	
+		                }
+		            }
+	            }
+			} catch(SQLException ex) {
+				log.error(ex);
+			}
+			return meta;
+		} else if(macro.indexOf("now")==0) {
+			return DatabaseWriter.formatTimeStamp(current);
+		} else if(macro.indexOf("past_hour")==0) {
+			return DatabaseWriter.formatTimeStamp(current-3600*1000L);
+		}
+		String[] tableList = dbc.findTableName(macro,current,current);
+		return tableList[0];
+	}
+
+	public static String getContents(File aFile) {
+        StringBuffer contents = new StringBuffer();    
+        try {
+        	BufferedReader input =  new BufferedReader(new FileReader(aFile));
+        	try {
+        		String line = null; //not declared within while loop
+        		while (( line = input.readLine()) != null){
+        			contents.append(line);
+        			contents.append(System.getProperty("line.separator"));
+        		}
+        	} finally {
+        		input.close();
+        	}
+        } catch (IOException ex){
+        	ex.printStackTrace();
+        }    
+        return contents.toString();
+    }
+
+	public void process(String table, String query) {
+		ResultSet rs = null;
+	    long start = current;
+	    long end = current;
+        
+		String cluster = System.getProperty("CLUSTER");
+		if(cluster==null) {
+			cluster="unknown";
+		}
+	    DatabaseWriter db = new DatabaseWriter(cluster);
+			    // Find the last aggregated value from table
+			    String[] tmpList = dbc.findTableName(table,start,end);
+			    String timeTest = "select timestamp from "+tmpList[0]+" order by timestamp desc limit 1";
+			    try {
+					rs = db.query(timeTest);
+				    while(rs.next()) {
+				    	start=rs.getTimestamp(1).getTime();
+				    	end=start;
+				    }
+			    } catch (SQLException e) {
+					// TODO Auto-generated catch block
+					e.printStackTrace();
+				}
+			    // Transform table names
+                HashMap<String, String> macroList = findMacros(query);
+                Iterator<String> macroKeys = macroList.keySet().iterator();
+                while(macroKeys.hasNext()) {
+                	String mkey = macroKeys.next();
+                	log.debug("replacing:"+mkey+" with "+macroList.get(mkey));
+			    	query = query.replace("["+mkey+"]", macroList.get(mkey));
+                }
+				log.info(query);
+                db.execute(query);
+            db.close();
+	}
+
+    public static void main(String[] args) {
+        loader=new PidFile(System.getProperty("CLUSTER")+"Aggregator");
+    	dbc = new DatabaseConfig();    	
+    	String queries = Aggregator.getContents(new File(System.getenv("CHUKWA_CONF_DIR")+File.separator+"aggregator.sql"));
+    	String[] query = queries.split("\n");
+    	for(int i=0;i<query.length;i++) {
+    		    int startOffset = query[i].indexOf("[")+1;
+    		    int endOffset = query[i].indexOf("]");
+    		    if(query[i].equals("")) {
+    		    } else if(startOffset==-1 || endOffset==-1) {
+    		    	log.error("Unable to extract table name from query:"+query[i]);
+    		    } else if(query[i].indexOf("#")==0) {
+    		    	log.debug("skipping: "+query[i]);
+    		    } else {
+    		    	String table = query[i].substring(startOffset, endOffset);
+    		    	Aggregator dba = new Aggregator();
+    		    	dba.process(table, query[i]);
+    		    }
+        }
+        loader.clean();
+    }
+
+}

+ 101 - 24
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/Consolidator.java → src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/Consolidator.java

@@ -16,24 +16,31 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.chukwa.extraction;
+package org.apache.hadoop.chukwa.database;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.chukwa.inputtools.mdl.DataConfig;
-import org.apache.hadoop.chukwa.util.ExceptionUtil;
 import org.apache.hadoop.chukwa.util.DatabaseWriter;
+import org.apache.hadoop.chukwa.util.ExceptionUtil;
+import org.apache.hadoop.chukwa.util.PidFile;
 
-import java.sql.*;
+import java.sql.SQLException;
+import java.sql.ResultSet;
 import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.sql.ResultSetMetaData;
 import java.text.SimpleDateFormat;
 
 public class Consolidator extends Thread {
-	private Log log = LogFactory.getLog(Consolidator.class);
+	private DatabaseConfig dbc = new DatabaseConfig();
+
+	private static Log log = LogFactory.getLog(Consolidator.class);
 	private String table = null;
-	private String jdbc = null;
 	private int[] intervals;
+    private static PidFile loader=null;
+
 	public Consolidator(String table, String intervalString) {
 		super(table);
 		try {
@@ -53,31 +60,64 @@ public class Consolidator extends Thread {
 		ResultSet rs = null;
 		String[] columns;
 		int[] columnsType;
-        String groupBy = "";
+		String groupBy = "";
         
 		for(int interval : intervals) {
 			// Start reducing from beginning of time;
 			Calendar aYearAgo = Calendar.getInstance();
-			aYearAgo.set(2008, 12, 30, 0, 0, 0);
+			aYearAgo.set(2008, 1, 1, 0, 0, 0);
 
 			long start = aYearAgo.getTimeInMillis();  //starting from 2008/01/01
 			long end = start + (interval*60000);
-		    log.debug("start time: "+start);
-		    log.debug("end time: "+end);
+			log.debug("start time: "+start);
+			log.debug("end time: "+end);
 			Calendar now = Calendar.getInstance();
-			DatabaseWriter db = new DatabaseWriter();
+			String cluster = System.getProperty("CLUSTER");
+			if(cluster==null) {
+				cluster="unknown";
+			}
+			DatabaseWriter db = new DatabaseWriter(cluster);
 			String fields = null;
 			String dateclause = null;
 			boolean emptyPrimeKey = false;
-			log.debug("Consolidate for "+interval+" minutes interval.");
-			String table = this.table+"_"+interval;
+			log.info("Consolidate for "+interval+" minutes interval.");
+			
+			String[] tmpTable = dbc.findTableName(this.table, start, end);
+			String table = tmpTable[0];
+			String sumTable="";
+			if(interval==5) {
+				long partition=now.getTime().getTime() / DatabaseConfig.WEEK;
+				StringBuilder stringBuilder = new StringBuilder();
+				stringBuilder.append(this.table);
+				stringBuilder.append("_");
+				stringBuilder.append(partition);
+				stringBuilder.append("_week");
+				table=stringBuilder.toString();
+				long partition2=now.getTime().getTime() / DatabaseConfig.MONTH;
+				sumTable =this.table+"_"+partition2+"_month";
+			} else if(interval==30) {
+				long partition=now.getTime().getTime() / DatabaseConfig.MONTH;
+				table=this.table+"_"+partition+"_month";				
+				long partition2=now.getTime().getTime() / DatabaseConfig.QUARTER;
+				sumTable =this.table+"_"+partition2+"_month";
+			} else if(interval==180) {
+				long partition=now.getTime().getTime() / DatabaseConfig.QUARTER;
+				table=this.table+"_"+partition+"_quarter";
+				long partition2=now.getTime().getTime() / DatabaseConfig.YEAR;
+				sumTable =this.table+"_"+partition2+"_month";
+			} else if(interval==720) {
+				long partition=now.getTime().getTime() / DatabaseConfig.YEAR;
+				table=this.table+"_"+partition+"_year";
+				long partition2=now.getTime().getTime() / DatabaseConfig.DECADE;
+				sumTable =this.table+"_"+partition2+"_month";
+			}
 			// Find the most recent entry
 			try {
-			    String query = "select * from "+table+" order by timestamp desc limit 1";
+			    String query = "select * from "+sumTable+" order by timestamp desc limit 1";
 	            log.debug("Query: "+query);
 	            rs = db.query(query);
 	            if(rs==null) {
-	          	    throw new SQLException("Table undefined.");
+	          	    throw new SQLException("Table is undefined.");
 	            }
 	            ResultSetMetaData rmeta = rs.getMetaData();
 	            boolean empty=true;
@@ -95,7 +135,7 @@ public class Consolidator extends Thread {
                 end = start + (interval*60000);
 			} catch (SQLException ex) {
 			    try {
-				    String query = "select * from "+this.table+" order by timestamp limit 1";
+				    String query = "select * from "+table+" order by timestamp limit 1";
 		            log.debug("Query: "+query);
 	                rs = db.query(query);
 	                if(rs.next()) {
@@ -127,13 +167,21 @@ public class Consolidator extends Thread {
 		    	    if(i==0) {
 		    		    fields=columns[i];
 	    	            if(columnsType[i]==java.sql.Types.VARCHAR) {
-	    	            	groupBy = " group by "+columns[i];
+	    	            	if(groupBy.equals("")) {
+	    	            	    groupBy = " group by "+columns[i];
+	    	            	} else {
+		    	            	groupBy = groupBy+","+columns[i];	    	            		
+	    	            	}
 	    	            }
 		    	    } else {
 		    		    if(columnsType[i]==java.sql.Types.VARCHAR || columnsType[i]==java.sql.Types.TIMESTAMP) {
 		    	            fields=fields+","+columns[i];
 		    	            if(columnsType[i]==java.sql.Types.VARCHAR) {
-		    	            	groupBy = " group by "+columns[i];
+		    	            	if(groupBy.equals("")) {
+		    	            	    groupBy = " group by "+columns[i];
+		    	            	} else {
+		    	            	    groupBy = groupBy+","+columns[i];		    	            		
+		    	            	}
 		    	            }
 		    		    } else {
 		    	            fields=fields+",AVG("+columns[i]+") as "+columns[i];
@@ -148,32 +196,61 @@ public class Consolidator extends Thread {
             	emptyPrimeKey = true;
             }
 			long previousStart = start;
+			long partition = 0;
+			String timeWindowType="week";
         	while(end < now.getTimeInMillis()-(interval*2*60000)) {
 			    // Select new data sample for the given intervals
 			    if(interval == 5) {
-				    table=this.table;
+			    	timeWindowType="month";
+					partition = start / DatabaseConfig.MONTH;
 			    } else if(interval == 30) {
-				    table=this.table+"_5";				
-			    } else if(interval == 120) {
-				    table=this.table+"_30";
+			    	timeWindowType="quarter";
+					partition = start / DatabaseConfig.QUARTER;
+			    } else if(interval == 180) {
+			    	timeWindowType="year";
+					partition = start / DatabaseConfig.YEAR;
+			    } else if(interval == 720) {
+			    	timeWindowType="decade";
+					partition = start / DatabaseConfig.DECADE;
 			    }
 	            SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
 			    String startS = formatter.format(start);
 			    String endS = formatter.format(end);
 			    dateclause = "Timestamp >= '"+startS+"' and Timestamp <= '"+endS+"'";
 			    if(emptyPrimeKey) {
-			    	groupBy = "group by "+dateclause;
+			    	groupBy = " group by FLOOR(UNIX_TIMESTAMP(TimeStamp)/"+interval*60+")";
 			    }
-				String query = "insert ignore into "+this.table+"_"+interval+" (select "+fields+" from "+table+" where "+dateclause+groupBy+")";
+				String query = "replace into "+this.table+"_"+partition+"_"+timeWindowType+" (select "+fields+" from "+table+" where "+dateclause+groupBy+")";
 				log.debug(query);
                 db.execute(query);
-                db.close();
         		if(previousStart == start) {
         			start = start + (interval*60000);
         			end = start + (interval*60000);
             		previousStart = start;
         		}
         	}
+            db.close();
 		}
 	}
+
+    public static void main(String[] args) {
+        DataConfig mdl = new DataConfig();
+        loader=new PidFile(System.getProperty("CLUSTER")+"Consolidator");
+        HashMap<String, String> tableNames = (HashMap<String, String>) mdl.startWith("consolidator.table.");
+        try {
+                Iterator<String> ti = (tableNames.keySet()).iterator();
+                while(ti.hasNext()) {
+                        String table = ti.next();
+                String interval=mdl.get(table);
+                table = table.substring(19);
+                        log.info("Summarizing table:"+table);
+                Consolidator dbc = new Consolidator(table, interval);
+                dbc.run();
+                }
+        } catch (NullPointerException e) {
+                log.error("Unable to summarize database.");
+                log.error("Error:"+ExceptionUtil.getStackTrace(e));
+        }
+        loader.clean();
+    }
 }

+ 106 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/DataExpiration.java

@@ -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.database;
+
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.chukwa.util.DatabaseWriter;
+
+public class DataExpiration {
+	private static DatabaseConfig dbc = null;
+	private static Log log = LogFactory.getLog(DataExpiration.class);		
+	public DataExpiration() {
+    	if(dbc==null) {
+    	    dbc = new DatabaseConfig();
+    	}
+    }
+	public void dropTables(long start, long end) {
+		String cluster = System.getProperty("CLUSTER");
+		if(cluster==null) {
+			cluster="unknown";
+		}
+		DatabaseWriter dbw = new DatabaseWriter(cluster);
+		try {
+			HashMap<String, String> dbNames = dbc.startWith("report.db.name.");
+			Iterator<String> ki = dbNames.keySet().iterator();
+			while(ki.hasNext()) {
+				String name = ki.next();
+				String tableName = dbNames.get(name);
+				String[] tableList = dbc.findTableName(tableName, start, end);
+				for(String tl : tableList) {
+					log.debug("table name: "+tableList[0]);
+					try {
+						String[] parts = tl.split("_");
+						int partition = Integer.parseInt(parts[parts.length-2]);
+						String table = "";
+						for(int i=0;i<parts.length-2;i++) {
+							if(i!=0) {
+								table=table+"_";
+							}
+							table=table+parts[i];
+						}
+						partition=partition-3;
+						String dropPartition="drop table if exists "+table+"_"+partition+"_"+parts[parts.length-1];
+						dbw.execute(dropPartition);
+						partition--;
+						dropPartition="drop table if exists "+table+"_"+partition+"_"+parts[parts.length-1];
+						dbw.execute(dropPartition);
+					} catch(NumberFormatException e) {
+						log.error("Error in parsing table partition number, skipping table:"+tableList[0]);
+					} catch(ArrayIndexOutOfBoundsException e) {
+						log.debug("Skipping table:"+tableList[0]+", because it has no partition configuration.");
+					}
+				}
+			}
+		} catch(Exception e) {
+			e.printStackTrace();
+		}		
+	}
+	
+	public static void usage() {
+		System.out.println("DataExpiration usage:");
+		System.out.println("java -jar chukwa-core.jar org.apache.hadoop.chukwa.DataExpiration <date> <time window size>");
+		System.out.println("     date format: YYYY-MM-DD");
+		System.out.println("     time window size: 7, 30, 91, 365");		
+	}
+	
+	public static void main(String[] args) {
+		DataExpiration de = new DataExpiration();
+		long now = (new Date()).getTime();
+		long start = now;
+		long end = now;
+		if(args.length==2) {
+			SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd");
+			try {
+				start = sdf.parse(args[0]).getTime();				
+				end = start + (Long.parseLong(args[1])*1440*60*1000L);
+				de.dropTables(start, end);				
+			} catch(Exception e) {
+				usage();
+			}
+		} else {
+			usage();
+		}
+    }
+}

+ 244 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/DatabaseConfig.java

@@ -0,0 +1,244 @@
+/*
+ * 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.database;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import java.util.*;
+
+public class DatabaseConfig {
+    private Configuration config = null;
+	public final static long CENTURY=36500*24*60*60*1000L;
+    public final static long DECADE=3650*24*60*60*1000L;
+    public final static long YEAR=365*24*60*60*1000L;
+    public final static long QUARTER=91250*24*60*60L;
+    public final static long MONTH=30*24*60*60*1000L;
+	public final static long WEEK=7*24*60*60*1000L;
+	public final static long DAY=24*60*60*1000L;
+
+    public DatabaseConfig(String path) {
+        Path fileResource = new Path(path);
+        config = new Configuration();
+        config.addResource(fileResource);
+    }
+    public DatabaseConfig() {
+        Path fileResource = new Path(System.getenv("DATACONFIG"));
+        config = new Configuration();
+        config.addResource(fileResource);
+    }
+
+    public String get(String key) {
+        return config.get(key);
+    }
+    public void put(String key, String value) {
+        this.config.set(key, value);
+    }
+    public Iterator<?> iterator() {
+        return this.config.iterator();
+    }
+    public HashMap<String, String> startWith(String key) {
+        HashMap<String, String> transformer = new HashMap<String, String>();
+        Iterator<?> entries = config.iterator();
+        while(entries.hasNext()) {
+           String entry = entries.next().toString();
+           if(entry.startsWith(key)) {
+               String[] metrics = entry.split("=");
+               transformer.put(metrics[0],metrics[1]);
+           }
+        }
+        return transformer;
+    }    
+    public String[] findTableName(String tableName, long start, long end) {
+    	String[] tableNames = null;
+    	String tableType = "_week";
+		long now = (new Date()).getTime();
+		long timeWindow = end - start;
+		long partitionSize=WEEK;
+		boolean fallback=true;
+		
+		if(config.get("consolidator.table."+tableName)==null) {
+			tableNames = new String[1];
+			tableNames[0]=tableName;
+			return tableNames;
+		}
+		
+		if(timeWindow<=0) {
+			timeWindow=1;			
+		}
+		if(timeWindow > DECADE) {
+			tableType = "_century";
+			partitionSize=CENTURY;
+		} else if(timeWindow > YEAR) {
+			tableType = "_decade";
+			partitionSize=DECADE;
+		} else if(timeWindow > QUARTER) {
+			tableType = "_year";
+			partitionSize=YEAR;			
+		} else if(timeWindow > MONTH) {
+			tableType = "_quarter";
+			partitionSize=QUARTER;
+		} else if(timeWindow > WEEK) {
+			tableType = "_month";
+			partitionSize=MONTH;
+		} else {
+			tableType = "_week";
+			partitionSize=WEEK;
+		}
+
+		long currentPartition = now / partitionSize;
+		long startPartition = start / partitionSize;
+		long endPartition = end / partitionSize;
+		while(fallback && partitionSize!=CENTURY*100) {
+			// Check if the starting date is in the far distance from current time.  If it is, use down sampled data.
+			if(startPartition + 2 < currentPartition) {
+				fallback=true;
+			    if(partitionSize==DAY) {
+				    tableType = "_week";
+				    partitionSize=WEEK;
+			    } else if(partitionSize==WEEK) {
+				    tableType = "_month";
+				    partitionSize=MONTH;
+			    } else if(partitionSize==MONTH) {
+				    tableType = "_year";
+				    partitionSize=YEAR;
+			    } else if(partitionSize==YEAR) {
+					tableType = "_decade";
+					partitionSize=DECADE;				
+				} else if(partitionSize==DECADE) {
+					tableType = "_century";
+					partitionSize=CENTURY;
+				} else {
+					partitionSize=100*CENTURY;
+				}
+				currentPartition = now / partitionSize;
+				startPartition = start / partitionSize;
+				endPartition = end / partitionSize;
+			} else {
+				fallback=false;
+			}
+		}
+
+		if(startPartition!=endPartition) {
+			int delta = (int) (endPartition-startPartition);
+			tableNames=new String[delta+1];
+			for(int i=0;i<=delta;i++) {
+				long partition = startPartition+(long)i;
+				tableNames[i]=tableName+"_"+partition+tableType;
+			}
+		} else {
+			tableNames=new String[1];
+			tableNames[0]=tableName+"_"+startPartition+tableType;
+		}
+    	return tableNames;
+    }
+    public String[] findTableNameForCharts(String tableName, long start, long end) {
+    	String[] tableNames = null;
+    	String tableType = "_week";
+		long now = (new Date()).getTime();
+		long timeWindow = end - start;
+		if(timeWindow>60*60*1000) {
+		    timeWindow = timeWindow + 1;
+		}
+		long partitionSize=WEEK;
+		boolean fallback=true;
+		
+		if(config.get("consolidator.table."+tableName)==null) {
+			tableNames = new String[1];
+			tableNames[0]=tableName;
+			return tableNames;
+		}
+		
+		if(timeWindow<=0) {
+			timeWindow=1;			
+		}
+		if(timeWindow > YEAR) {
+			tableType = "_century";
+			partitionSize=CENTURY;			
+		} else if(timeWindow > QUARTER) {
+			tableType = "_century";
+			partitionSize=CENTURY;
+		} else if(timeWindow > MONTH) {
+			tableType = "_decade";
+			partitionSize=DECADE;
+		} else if(timeWindow > WEEK) {
+			tableType = "_year";
+			partitionSize=YEAR;
+		} else if(timeWindow > DAY) {
+			tableType = "_quarter";
+			partitionSize=QUARTER;
+		} else if(timeWindow > 60*60*1000) {
+			tableType = "_month";
+			partitionSize=MONTH;			
+		} else {
+			tableType = "_week";
+			partitionSize = WEEK;
+		}
+
+		long currentPartition = now / partitionSize;
+		long startPartition = start / partitionSize;
+		long endPartition = end / partitionSize;
+		while(fallback && partitionSize!=DECADE*100) {
+			// Check if the starting date is in the far distance from current time.  If it is, use down sampled data.
+			if(startPartition + 2 < currentPartition) {
+				fallback=true;
+			    if(partitionSize==DAY) {
+				    tableType = "_month";
+				    partitionSize=MONTH;
+			    } else if(partitionSize==WEEK) {
+				    tableType = "_quarter";
+				    partitionSize=QUARTER;
+			    } else if(partitionSize==MONTH) {
+				    tableType = "_year";
+				    partitionSize=YEAR;
+			    } else if(partitionSize==YEAR) {
+					tableType = "_decade";
+					partitionSize=DECADE;				
+				} else {
+					partitionSize=CENTURY;
+				}
+				currentPartition = now / partitionSize;
+				startPartition = start / partitionSize;
+				endPartition = end / partitionSize;
+			} else {
+				fallback=false;
+			}
+		}
+
+		if(startPartition!=endPartition) {
+			int delta = (int) (endPartition-startPartition);
+			tableNames=new String[delta+1];
+			for(int i=0;i<=delta;i++) {
+				long partition = startPartition+(long)i;
+				tableNames[i]=tableName+"_"+partition+tableType;
+			}
+		} else {
+			tableNames=new String[1];
+			tableNames[0]=tableName+"_"+startPartition+tableType;
+		}
+    	return tableNames;
+    }
+    
+    public static void main(String[] args) {
+    	DatabaseConfig dbc = new DatabaseConfig();
+    	String[] names = dbc.findTableName("system_metrics",1216140020000L,1218645620000L);
+    	for(String n: names) {
+    		System.out.println("name:"+n);
+    	}
+    }
+}

+ 159 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/MetricsAggregation.java

@@ -0,0 +1,159 @@
+package org.apache.hadoop.chukwa.database;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+public class MetricsAggregation
+{
+	 private static Log log = LogFactory.getLog(MetricsAggregation.class);
+	 private static Connection conn = null;    
+     private static Statement stmt = null; 
+     private static ResultSet rs = null; 
+     private static DatabaseConfig mdlConfig;
+     
+	/**
+	 * @param args
+	 * @throws SQLException 
+	 */
+	public static void main(String[] args) throws SQLException
+	{
+	       mdlConfig = new DatabaseConfig();
+		
+	       // Connect to the database
+	       String jdbc_url = System.getenv("JDBC_URL_PREFIX")+mdlConfig.get("jdbc.host")+"/"+mdlConfig.get("jdbc.db");
+	       if(mdlConfig.get("jdbc.user")!=null) {
+	           jdbc_url = jdbc_url + "?user=" + mdlConfig.get("jdbc.user");
+	           if(mdlConfig.get("jdbc.password")!=null) {
+	               jdbc_url = jdbc_url + "&password=" + mdlConfig.get("jdbc.password");
+	           }
+	       }
+	       try {
+	           // The newInstance() call is a work around for some
+	           // broken Java implementations
+                   String jdbcDriver = System.getenv("JDBC_DRIVER");
+	           Class.forName(jdbcDriver).newInstance();
+	           log.info("Initialized JDBC URL: "+jdbc_url);
+	       } catch (Exception ex) {
+	           // handle the error
+	    	   ex.printStackTrace();
+	           log.error(ex,ex);
+	       }
+	       try {
+	           conn = DriverManager.getConnection(jdbc_url);
+	       } catch (SQLException ex) 
+	       {
+	    	   ex.printStackTrace();
+	           log.error(ex,ex);
+	       }      
+	       
+	       // get the latest timestamp for aggregation on this table
+		   // Start = latest
+	       
+	      
+	       
+	       SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
+	        
+	       long start = System.currentTimeMillis() - (1000*60*60*24);
+	       long end = System.currentTimeMillis() - (1000*60*10);
+	       // retrieve metadata for cluster_system_metrics
+	       DatabaseConfig dbConf = new DatabaseConfig();
+	       String[] tables = dbConf.findTableName("cluster_system_metrics_2018_week", start, end);
+	       for(String table: tables)
+	       {
+	    	   System.out.println("Table to aggregate per Ts: " + table);
+	    	   stmt = conn.createStatement();
+	    	   rs = stmt.executeQuery("select table_ts from aggregation_admin_table where table_name=\"" 
+	    			   + table + "\"");
+			   if (rs.next())
+			   {
+				   start = rs.getLong(1);
+			   }
+			   else
+			   {
+				   start = 0;
+			   }
+			   
+			   end = start + (1000*60*60*1); // do 1 hour aggregation max 
+			   long now = System.currentTimeMillis();
+			   now = now - (1000*60*10); // wait for 10 minutes
+			   end = Math.min(now, end);
+		     
+			   // TODO REMOVE DEBUG ONLY!
+			   end = now;
+			   
+			   System.out.println("Start Date:" + new Date(start));
+			   System.out.println("End Date:" + new Date(end));
+			   
+		       DatabaseMetaData dbm = conn.getMetaData ();
+		       rs = dbm.getColumns ( null,null,table, null);
+		      	
+		       List<String> cols = new ArrayList<String>();
+		       while (rs.next ())
+		       {
+		          	String s = rs.getString (4); // 4 is column name, 5 data type etc. 
+		          	System.out.println ("Name: " + s);
+		          	int type = rs.getInt(5);
+		          	if (type == java.sql.Types.VARCHAR)
+		          	{
+		          		System.out.println("Type: Varchar " + type);
+		          	}
+		          	else
+		          	{
+		          		cols.add(s);
+		          		System.out.println("Type: Number " + type);
+		          	}
+		       }// end of while.
+		       
+		       // build insert into from select query
+		       String initTable = table.replace("cluster_", "");
+		       StringBuilder sb0 = new StringBuilder();
+		       StringBuilder sb = new StringBuilder();
+		       sb0.append("insert into ").append(table).append(" (");
+		       sb.append(" ( select ");
+		       for (int i=0;i<cols.size();i++)
+		       {
+		    	   sb0.append(cols.get(i));
+		    	   sb.append("avg(").append(cols.get(i)).append(") ");
+		    	   if (i< cols.size()-1)
+		    	   {
+		    		   sb0.append(",");
+		    		   sb.append(",");
+		    	   }
+		       }
+			   sb.append(" from ").append(initTable);
+			   sb.append(" where timestamp between \"");
+			   sb.append(formatter.format(start));
+			   sb.append("\" and \"").append(formatter.format(end));
+			   sb.append("\" group by timestamp  )");
+			  
+		        
+			   // close fields
+			   sb0.append(" )").append(sb);
+			   System.out.println(sb0.toString());
+			   
+			   // run query
+			   conn.setAutoCommit(false);
+			   stmt = conn.createStatement();
+			   stmt.execute(sb0.toString());
+			   
+			   // update last run
+			   stmt = conn.createStatement();
+			   stmt.execute("insert into aggregation_admin_table set table_ts=\"" +  formatter.format(end) +
+					   "\" where table_name=\"" + table + "\"");
+			   conn.commit();
+	       }
+	
+	}
+
+}

+ 126 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/database/TableCreator.java

@@ -0,0 +1,126 @@
+/*
+ * 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.database;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.chukwa.util.DatabaseWriter;
+
+public class TableCreator {
+	private static DatabaseConfig dbc = null;
+	private static Log log = LogFactory.getLog(TableCreator.class);		
+	public TableCreator() {
+    	if(dbc==null) {
+    	    dbc = new DatabaseConfig();
+    	}
+    }
+	public void createTables() {
+		long now = (new Date()).getTime();
+        createTables(now,now);		
+	}
+	public void createTables(long start, long end) {
+		String cluster = System.getProperty("CLUSTER");
+		if(cluster==null) {
+			cluster="unknown";
+		}
+		DatabaseWriter dbw = new DatabaseWriter(cluster);
+		try {
+			HashMap<String, String> dbNames = dbc.startWith("report.db.name.");
+			Iterator<String> ki = dbNames.keySet().iterator();
+			while(ki.hasNext()) {
+				String name = ki.next();
+				String tableName = dbNames.get(name);
+				String[] tableList = dbc.findTableName(tableName, start, end);
+				log.debug("table name: "+tableList[0]);
+				try {
+				    String[] parts = tableList[0].split("_");
+				    int partition = Integer.parseInt(parts[parts.length-2]);
+				    String table = "";
+				    for(int i=0;i<parts.length-2;i++) {
+				    	if(i!=0) {
+				    		table=table+"_";
+				    	}
+				    	table=table+parts[i];
+				    }
+				    String query = "show create table "+table+"_template;";
+				    ResultSet rs = dbw.query(query);
+                    while(rs.next()) {				    
+                    	log.debug("table schema: "+rs.getString(2));
+                    	query=rs.getString(2);
+                    	log.debug("template table name:"+table+"_template");
+                    	log.debug("replacing with table name:"+table+"_"+partition+"_"+parts[parts.length-1]);
+                    	log.debug("creating table: "+query);
+                    	String createPartition=query.replaceFirst(table+"_template", table+"_"+partition+"_"+parts[parts.length-1]);
+                    	createPartition=createPartition.replaceFirst("TABLE","TABLE IF NOT EXISTS");
+                    	dbw.execute(createPartition);
+                    	partition++;
+                    	createPartition=query.replaceFirst(table+"_template", table+"_"+partition+"_"+parts[parts.length-1]);
+                    	createPartition=createPartition.replaceFirst("TABLE","TABLE IF NOT EXISTS");
+                    	dbw.execute(createPartition);
+                    	partition++;
+                    	createPartition=query.replaceFirst(table+"_template", table+"_"+partition+"_"+parts[parts.length-1]);
+                    	createPartition=createPartition.replaceFirst("TABLE","TABLE IF NOT EXISTS");
+                    	dbw.execute(createPartition);
+                    }
+				} catch(NumberFormatException e) {
+					log.error("Error in parsing table partition number, skipping table:"+tableList[0]);
+				} catch(ArrayIndexOutOfBoundsException e) {
+					log.debug("Skipping table:"+tableList[0]+", because it has no partition configuration.");
+				} catch(SQLException e) {
+					
+				}
+			}
+		} catch(Exception e) {
+			e.printStackTrace();
+		}		
+	}
+
+	public static void usage() {
+		System.out.println("TableCreator usage:");
+		System.out.println("java -jar chukwa-core.jar org.apache.hadoop.chukwa.TableCreator <date> <time window size>");
+		System.out.println("     date format: YYYY-MM-DD");
+		System.out.println("     time window size: 7, 30, 91, 365, 3650");
+	}
+	
+	public static void main(String[] args) {
+		TableCreator tc = new TableCreator();
+        if(args.length==2) {
+        	try {
+        		SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd");
+        		long start = sdf.parse(args[0]).getTime();
+        		long end = start + (Long.parseLong(args[1])*1440*60*1000L);
+        		tc.createTables(start, end);
+        	} catch(Exception e) {
+        		System.out.println("Invalid date format or time window size.");
+        		e.printStackTrace();
+				usage();
+        	}
+		} else {
+			tc.createTables();
+		}
+
+    }
+}

+ 4 - 4
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/ChunkQueue.java

@@ -31,16 +31,16 @@ public interface ChunkQueue extends ChunkReceiver
 {
   /**
    *  Add a chunk to the queue, blocking if queue is full.
-   * @param event
+   * @param chunk
    * @throws InterruptedException if thread is interrupted while blocking
    */
-	public void add(Chunk event) throws InterruptedException;
+	public void add(Chunk chunk) throws InterruptedException;
 	
 	/**
-	 * Return at least one, and no more than count, Chunks into events.
+	 * Return at least one, and no more than count, Chunks into chunks.
 	 * Blocks if queue is empty.
 	 */
-	public void collect(List<Chunk> events,int count) throws InterruptedException;
+	public void collect(List<Chunk> chunks,int count) throws InterruptedException;
 	
 	/**
 	 * Return an approximation of the number of chunks in the queue currently.

+ 22 - 7
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/DataFactory.java

@@ -31,6 +31,7 @@ public class DataFactory
 {
   static Logger log = Logger.getLogger(DataFactory.class);
 	static final int QUEUE_SIZE_KB = 10 * 1024;
+	static final String COLLECTORS_FILENAME = "collectors";
 	private static DataFactory dataFactory = null;
 	private ChunkQueue chunkQueue = new MemLimitQueue(QUEUE_SIZE_KB * 1024);
 
@@ -54,14 +55,28 @@ public class DataFactory
 	 * @return empty list if file does not exist
 	 * @throws IOException on other error
 	 */
-	public Iterator<String> getCollectors() throws IOException
+	public Iterator<String> getCollectorURLs() throws IOException
 	{
-	  String chukwaHome = System.getenv("CHUKWA_HOME");
-	  if (chukwaHome == null){
-	    chukwaHome = ".";
-	  }
-	  log.info("setting up collectors file: " + chukwaHome + "/conf/collectors");
-		File collectors = new File(chukwaHome + "/conf/collectors");
+		  String chukwaHome = System.getenv("CHUKWA_HOME");
+		  if (chukwaHome == null){
+			  chukwaHome = ".";
+		  }
+
+		  if(!chukwaHome.endsWith("/"))
+		  {  chukwaHome = chukwaHome + File.separator; }
+		  log.info("Config - System.getenv(\"CHUKWA_HOME\"): [" + chukwaHome + "]" );
+
+		  String chukwaConf = System.getenv("CHUKWA_CONF_DIR");    
+		  if (chukwaConf == null)
+		  {
+			  chukwaConf = chukwaHome + "conf" + File.separator;
+		  }
+
+		  log.info("Config - System.getenv(\"chukwaConf\"): [" + chukwaConf + "]" );
+		  
+	  log.info("setting up collectors file: " + chukwaConf + 
+	      File.separator + COLLECTORS_FILENAME);
+		File collectors = new File(chukwaConf + File.separator + "collectors");
 		try{
 		  return new RetryListOfCollectors(collectors, 1000 * 15);//time is ms between tries
 		} catch(java.io.IOException e) {

+ 7 - 1
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/Adaptor.java

@@ -44,7 +44,7 @@ public interface Adaptor
    * @param offset the stream offset of the first byte sent by this adaptor
    * @throws AdaptorException
    */
-	public void start(String type, String status, long offset, ChunkReceiver dest) throws AdaptorException;
+	public void start(long adaptorID, String type, String status, long offset, ChunkReceiver dest) throws AdaptorException;
 	
 	/**
 	 * Return the adaptor's state
@@ -54,6 +54,12 @@ public interface Adaptor
 	 */
 	public String getCurrentStatus() throws AdaptorException;
 	public String getType();
+	
+	/**
+	 * Return the stream name
+	 * @return Stream name as a string
+	 */
+	public String getStreamName();
 	/**
 	 * Signals this adaptor to come to an orderly stop.
 	 * The adaptor ought to push out all the data it can

+ 45 - 12
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/ExecAdaptor.java

@@ -17,18 +17,32 @@
  */
 
 package org.apache.hadoop.chukwa.datacollection.adaptor;
-
 import org.apache.hadoop.chukwa.ChunkImpl;
 import org.apache.hadoop.chukwa.datacollection.ChunkReceiver;
 import org.apache.hadoop.chukwa.inputtools.plugin.ExecPlugin;
+import org.apache.log4j.Logger;
 import org.apache.log4j.helpers.ISO8601DateFormat;
 import org.json.JSONException;
 import org.json.JSONObject;
 import java.util.*;
 
+/**
+ * Runs a command inside chukwa.  Takes as params the interval 
+ * in seconds at which to run the command, and the path and args
+ * to execute.
+ * 
+ * Interval is optional, and defaults to 5 seconds.
+ * 
+ * Example usage:  
+ * add org.apache.hadoop.chukwa.datacollection.adaptor.ExecAdaptor Ps 2 /bin/ps aux 0
+ * 
+ */
 public class ExecAdaptor extends ExecPlugin implements Adaptor {
 
   static final boolean FAKE_LOG4J_HEADER = true;
+  static final boolean SPLIT_LINES = false;
+  protected long adaptorID = 0;
+  static Logger log =Logger.getLogger(ExecAdaptor.class);
    
   class RunToolTask extends TimerTask {
     public void run() {
@@ -56,15 +70,19 @@ public class ExecAdaptor extends ExecPlugin implements Adaptor {
           data = stdout.getBytes();
         }
  
-        ArrayList<Integer> carriageReturns = new  ArrayList<Integer>();
-        for(int i = 0; i < data.length ; ++i)
-          if(data[i] == '\n')
-            carriageReturns.add(i);
-        
         sendOffset += data.length;
         ChunkImpl c = new ChunkImpl(ExecAdaptor.this.type,
             "results from " + cmd, sendOffset , data, ExecAdaptor.this);
-        c.setRecordOffsets(carriageReturns);
+        
+        if(SPLIT_LINES) {
+          ArrayList<Integer> carriageReturns = new  ArrayList<Integer>();
+          for(int i = 0; i < data.length ; ++i)
+            if(data[i] == '\n')
+              carriageReturns.add(i);
+          
+          c.setRecordOffsets(carriageReturns);
+        }  //else we get default one record
+        
         dest.add(c);
       } catch(JSONException e ) {
         //FIXME: log this somewhere
@@ -89,9 +107,12 @@ public class ExecAdaptor extends ExecPlugin implements Adaptor {
   
   @Override
   public String getCurrentStatus() throws AdaptorException {
-    return cmd;
+    return type + " " + period + " " + cmd + " " + sendOffset;
   }
 
+  public String getStreamName() {
+	  return cmd;
+  }
   @Override
   public void hardStop() throws AdaptorException {
     super.stop();
@@ -110,10 +131,22 @@ public class ExecAdaptor extends ExecPlugin implements Adaptor {
   }
 
   @Override
-  public void start(String type, String status, long offset, ChunkReceiver dest)
-      throws AdaptorException
-  {
-    cmd = status;
+  public void start(long adaptorID, String type, String status, long offset, ChunkReceiver dest)
+      throws AdaptorException  {
+    
+    int spOffset = status.indexOf(' ');
+    if(spOffset > 0) {
+    try {
+      period = Integer.parseInt(status.substring(0, spOffset));
+      cmd = status.substring(spOffset + 1);
+    } catch(NumberFormatException e) {
+      log.warn("ExecAdaptor: sample interval " + status.substring(0, spOffset) + " can't be parsed");
+      cmd = status;
+      }
+   }
+    else
+      cmd = status;
+    this.adaptorID = adaptorID;
     this.type = type;
     this.dest = dest;
     this.sendOffset = offset;

+ 143 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/FileAdaptor.java

@@ -0,0 +1,143 @@
+/*
+ * 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.datacollection.adaptor;
+
+import org.apache.hadoop.chukwa.ChunkImpl;
+import org.apache.hadoop.chukwa.datacollection.ChunkReceiver;
+import org.apache.hadoop.chukwa.datacollection.agent.ChukwaAgent;
+import org.apache.hadoop.chukwa.util.RecordConstants; 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Logger;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.ArrayList;
+
+
+/**
+ * File Adaptor push small size file in one chunk to collector
+ */
+public class FileAdaptor  implements Adaptor
+{
+
+	static Logger log;
+
+	protected static Configuration conf = null;
+	private int attempts = 0;
+	
+	File toWatch;
+	/**
+	 * next PHYSICAL offset to read
+	 */
+	protected long fileReadOffset;
+	protected String type;
+	private ChunkReceiver dest;
+	protected RandomAccessFile reader = null;
+	protected long adaptorID;
+	
+	/**
+	 * The logical offset of the first byte of the file
+	 */
+	private long offsetOfFirstByte = 0;
+	
+	static {
+		log =Logger.getLogger(FileAdaptor.class);
+	}
+
+	public void start(long adaptorID, String type, String params, long bytes, ChunkReceiver dest) {
+	    //in this case params = filename 
+		log.info("adaptor id: "+adaptorID+" started file adaptor on file " + params);
+		this.adaptorID = adaptorID;
+	    this.type = type;
+	    this.dest = dest;
+	    this.attempts = 0;
+			  
+	    String[] words = params.split(" ");
+	    if(words.length > 1) {
+	        offsetOfFirstByte = Long.parseLong(words[0]);
+	        toWatch = new File(params.substring(words[0].length() + 1));
+	    } else {
+	        toWatch = new File(params);
+	    }
+	    try {
+	  		reader = new RandomAccessFile(toWatch, "r");
+	  		long bufSize = toWatch.length();
+			byte[] buf = new byte[(int) bufSize];
+			reader.read(buf);
+	        long fileTime = toWatch.lastModified();
+			int bytesUsed = extractRecords(dest, 0, buf, fileTime);
+	    } catch(Exception e) {
+	        e.printStackTrace();
+	    }
+		ChukwaAgent agent = ChukwaAgent.getAgent();
+		if (agent != null) {
+			agent.stopAdaptor(adaptorID, false);
+		} else {
+			log.info("Agent is null, running in default mode");
+		}
+		this.fileReadOffset= bytes;
+	}
+
+	/**
+	 * Do one last tail, and then stop
+	 * @see org.apache.hadoop.chukwa.datacollection.adaptor.Adaptor#shutdown()
+	 */
+	public long shutdown() throws AdaptorException {
+	  hardStop();
+	  return fileReadOffset + offsetOfFirstByte;
+	}
+	/**
+	 * Stop tailing the file, effective immediately.
+	 */
+	public void hardStop() throws AdaptorException {
+	}
+
+	public String getStreamName() {
+		return toWatch.getPath();
+	}
+	
+  /**
+   * Extract records from a byte sequence
+   * @param eq the queue to stick the new chunk[s] in
+   * @param buffOffsetInFile the byte offset in the stream at which buf[] begins
+   * @param buf the byte buffer to extract records from
+   * @return the number of bytes processed
+   * @throws InterruptedException
+   */
+  protected int extractRecords(ChunkReceiver eq, long buffOffsetInFile, byte[] buf, long fileTime) throws InterruptedException {
+    ChunkImpl chunk = new ChunkImpl(type, toWatch.getAbsolutePath(), buffOffsetInFile + buf.length,
+        buf, this);
+    String tags = chunk.getTags();
+    chunk.setTags(tags+" time=\""+fileTime+"\"");
+    eq.add(chunk);
+    return buf.length;
+  }
+
+  @Override
+  public String getType() {
+    return type;
+  }
+
+  @Override
+  public String getCurrentStatus() throws AdaptorException {
+    return type.trim() + " " + offsetOfFirstByte+ " " + toWatch.getPath() + " " + fileReadOffset;
+  }
+  
+}

+ 26 - 13
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/FileTailer.java

@@ -23,6 +23,9 @@ import java.util.concurrent.CopyOnWriteArrayList;
 
 import org.apache.hadoop.chukwa.datacollection.DataFactory;
 import org.apache.hadoop.chukwa.datacollection.ChunkQueue;
+import org.apache.hadoop.chukwa.datacollection.agent.ChukwaAgent;
+import org.apache.hadoop.conf.Configuration;
+import org.mortbay.log.Log;
 
 /**
  * A shared thread used by all FileTailingAdaptors. 
@@ -40,21 +43,32 @@ class FileTailer extends Thread {
   /**
    * How often to tail each file.
    */
-  int SAMPLE_PERIOD_MS = 1000* 2; //FIXME: should be configurable
+  int DEFAULT_SAMPLE_PERIOD_MS = 1000* 2;
+  int SAMPLE_PERIOD_MS = DEFAULT_SAMPLE_PERIOD_MS;
+  private static Configuration conf = null;
   
   FileTailer() {
-     eq = DataFactory.getInstance().getEventQueue();
+	if (conf == null) {
+	  ChukwaAgent agent = ChukwaAgent.getAgent();
+	  if (agent != null) {
+		conf = agent.getConfiguration();
+    	if (conf != null) {
+    	  SAMPLE_PERIOD_MS= conf.getInt("chukwaAgent.adaptor.context.switch.time", DEFAULT_SAMPLE_PERIOD_MS);
+        }
+	  }
+	}
+    eq = DataFactory.getInstance().getEventQueue();
      
-       //iterations are much more common than adding a new adaptor
-     adaptors = new CopyOnWriteArrayList<FileTailingAdaptor>();
+    //iterations are much more common than adding a new adaptor
+    adaptors = new CopyOnWriteArrayList<FileTailingAdaptor>();
 
-     this.setDaemon(true);
-     start();//start the file-tailing thread
+    this.setDaemon(true);
+    start();//start the file-tailing thread
   }
    
   //called by FileTailingAdaptor, only
    void startWatchingFile(FileTailingAdaptor f) {
-       adaptors.add(f);
+     adaptors.add(f);
    }
 
    //called by FileTailingAdaptor, only
@@ -64,7 +78,7 @@ class FileTailer extends Thread {
    
   public void run()  {
     try{
-      while(true) {
+      while(true) {    	  
         boolean shouldISleep = true;
         long startTime = System.currentTimeMillis();
         for(FileTailingAdaptor f: adaptors) {
@@ -72,13 +86,12 @@ class FileTailer extends Thread {
           shouldISleep &= !hasMoreData;
         }
         long timeToReadFiles = System.currentTimeMillis() - startTime;
-        assert timeToReadFiles >= 0 : " time shouldn't go backwards";
-        if(timeToReadFiles < SAMPLE_PERIOD_MS && shouldISleep)
-          Thread.sleep(SAMPLE_PERIOD_MS - timeToReadFiles+1);
+        if(timeToReadFiles < SAMPLE_PERIOD_MS || shouldISleep) {
+          Thread.sleep(SAMPLE_PERIOD_MS);
+        }
       }
+    } catch(InterruptedException e) {
     }
-    catch(InterruptedException e)
-    {}
   }
   
   

+ 164 - 50
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/FileTailingAdaptor.java

@@ -22,9 +22,13 @@ import org.apache.hadoop.chukwa.ChunkImpl;
 import org.apache.hadoop.chukwa.datacollection.ChunkReceiver;
 import org.apache.hadoop.chukwa.datacollection.adaptor.Adaptor;
 import org.apache.hadoop.chukwa.datacollection.adaptor.AdaptorException;
+import org.apache.hadoop.chukwa.datacollection.agent.ChukwaAgent;
+import org.apache.hadoop.chukwa.inputtools.plugin.metrics.Exec;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Logger;
 
 import java.io.*;
+import java.util.Timer;
 
 /**
  * An adaptor that repeatedly tails a specified file, sending the new bytes.
@@ -42,8 +46,12 @@ public class FileTailingAdaptor implements Adaptor
 	 * to the next. This way, we get quick response time for other files if one
 	 * file is growing rapidly.
 	 */
-	public static final int MAX_READ_SIZE = 128 * 1024;
-
+	public static final int DEFAULT_MAX_READ_SIZE = 128 * 1024 ;
+	public static int MAX_READ_SIZE = DEFAULT_MAX_READ_SIZE ;
+	public static int MAX_RETRIES = 300;
+	protected static Configuration conf = null;
+	private int attempts = 0;
+	
 	File toWatch;
 	/**
 	 * next PHYSICAL offset to read
@@ -51,6 +59,8 @@ public class FileTailingAdaptor implements Adaptor
 	protected long fileReadOffset;
 	protected String type;
 	private ChunkReceiver dest;
+	protected RandomAccessFile reader = null;
+	protected long adaptorID;
 	
 	/**
 	 * The logical offset of the first byte of the file
@@ -64,21 +74,24 @@ public class FileTailingAdaptor implements Adaptor
 		log =Logger.getLogger(FileTailingAdaptor.class);
 	}
 
-	public void start(String type, String params, long bytes, ChunkReceiver dest) {
-	  //in this case params = filename 
+	public void start(long adaptorID, String type, String params, long bytes, ChunkReceiver dest) {
+	    //in this case params = filename 
 		log.info("started file tailer on file " + params);
-	  this.type = type;
-	  this.dest = dest;
+		this.adaptorID = adaptorID;
+	    this.type = type;
+	    this.dest = dest;
+	    this.attempts = 0;
 			  
-	  String[] words = params.split(" ");
-	  if(words.length > 1) {
-	    offsetOfFirstByte = Long.parseLong(words[0]);
-	    toWatch = new File(params.substring(words[0].length() + 1));
-	  }
-	  else
-	    toWatch = new File(params);
+	    String[] words = params.split(" ");
+	    if(words.length > 1) {
+	        offsetOfFirstByte = Long.parseLong(words[0]);
+	        toWatch = new File(params.substring(words[0].length() + 1));
+	    } else {
+	        toWatch = new File(params);
+	    }
 	  
-		this.fileReadOffset= bytes - offsetOfFirstByte;
+	    
+		this.fileReadOffset= bytes;
 		tailer.startWatchingFile(this);
 	}
 
@@ -88,25 +101,38 @@ public class FileTailingAdaptor implements Adaptor
 	 */
 	public long shutdown() throws AdaptorException {
 	  try{
-	    tailFile(tailer.eq); // get tail end of file.
-	  } catch(InterruptedException e) {
-	    Thread.currentThread().interrupt();
+	    if(toWatch.exists()) {
+	    	int retry=0;
+	    	tailer.stopWatchingFile(this);
+			TerminatorThread lastTail = new TerminatorThread(this,tailer.eq);
+			lastTail.setDaemon(true);
+			lastTail.start();
+			while(lastTail.isAlive() && retry < 60) {
+				try {
+					log.info("Retry:"+retry);
+				    Thread.currentThread().sleep(1000);
+				    retry++;
+				} catch(InterruptedException ex) {
+				}
+			}
+	    }
+	  } finally {
+	    return fileReadOffset + offsetOfFirstByte;
 	  }
-		hardStop();//need to do this afterwards, so that offset stays visible during tailFile().
-		return fileReadOffset + offsetOfFirstByte;
+
 	}
 	/**
 	 * Stop tailing the file, effective immediately.
 	 */
 	public void hardStop() throws AdaptorException {
-    tailer.stopWatchingFile(this);
+        tailer.stopWatchingFile(this);
 	}
 
   /**
    * @see org.apache.hadoop.chukwa.datacollection.adaptor.Adaptor#getCurrentStatus()
    */
 	public String getCurrentStatus() {
-		return type + " " + offsetOfFirstByte+ " " + toWatch.getPath();
+		return type.trim() + " " + offsetOfFirstByte+ " " + toWatch.getPath() + " " + fileReadOffset;
 		// can make this more efficient using a StringBuilder
 	}
 
@@ -114,6 +140,10 @@ public class FileTailingAdaptor implements Adaptor
 		return "Tailer on " + toWatch;
 	}
 
+	public String getStreamName() {
+		return toWatch.getPath();
+	}
+	
 	/**
 	 * Looks at the tail of the associated file, adds some of it to event queue
 	 * This method is not thread safe. Returns true if there's more data in the
@@ -123,35 +153,119 @@ public class FileTailingAdaptor implements Adaptor
 	 */
 	public synchronized boolean tailFile(ChunkReceiver eq) throws InterruptedException {
     boolean hasMoreData = false;
-    try {
-      if(!toWatch.exists())
-        return false;  //no more data
-      
-    	RandomAccessFile reader = new RandomAccessFile(toWatch, "r");
-    	long len = reader.length();
-    	if (len > fileReadOffset) {
-    		reader.seek(fileReadOffset);
-    
-    		long bufSize = len - fileReadOffset;
-    		if (bufSize > MAX_READ_SIZE) {
-    			bufSize = MAX_READ_SIZE;
-    			hasMoreData = true;
-    		}
-    		byte[] buf = new byte[(int) bufSize];
-    		reader.read(buf);
-    		assert reader.getFilePointer() == fileReadOffset + bufSize : " event size arithmetic is broken: "
-    				+ " pointer is "
-    				+ reader.getFilePointer()
-    				+ " but offset is " + fileReadOffset + bufSize;
-    
-    		int bytesUsed = extractRecords(dest, fileReadOffset + offsetOfFirstByte, buf);
-    		fileReadOffset = fileReadOffset + bytesUsed;
-    	}
-    	reader.close();
-    } catch (IOException e) {
-    	log.warn("failure reading " + toWatch, e);
-    }
-    return hasMoreData;
+	    try {
+	        if(!toWatch.exists() && attempts>MAX_RETRIES) {
+	    	    log.warn("Adaptor|" + adaptorID +"| File does not exist: "+toWatch.getAbsolutePath()+", streaming policy expired.  File removed from streaming.");
+       			ChukwaAgent agent = ChukwaAgent.getAgent();
+    			if (agent != null) {
+    				agent.stopAdaptor(adaptorID, false);
+    			} else {
+    				log.info("Agent is null, running in default mode");
+    			}
+    		    tailer.stopWatchingFile(this);
+	    	    return false;
+	        } else if(!toWatch.exists()) {
+	        	log.warn("failed to stream data for: "+toWatch.getAbsolutePath()+", attempt: "+attempts+" of "+MAX_RETRIES);
+	        	attempts++;
+	            return false;  //no more data
+	        }
+	      	if (reader == null)
+	      	{
+	      		reader = new RandomAccessFile(toWatch, "r");
+	      		log.info("Adaptor|" + adaptorID + "|Opening the file for the first time|seek|" + fileReadOffset);
+	      	}
+	      	
+	      	long len = 0L;
+	    	try {
+		      	RandomAccessFile newReader = new RandomAccessFile(toWatch,"r");
+		    	len = reader.length();
+		    	long newLength = newReader.length();
+		      	if(newLength<len && fileReadOffset >= len) {
+		      		reader.close();
+		      		reader = newReader;
+		      		fileReadOffset=0L;
+		      		log.debug("Adaptor|" + adaptorID +"| File size mismatched, rotating: "+toWatch.getAbsolutePath());
+		      	} else {
+		      		try {
+		      		    newReader.close();
+		      		} catch(IOException e) {
+		      			// do nothing.
+		      		}
+		      	}
+	    	} catch(IOException e) {
+      			// do nothing, if file doesn't exist.	    		
+	    	}
+	    	if (len >= fileReadOffset) {
+	    		if(offsetOfFirstByte>fileReadOffset) {
+	    			// If the file rotated, the recorded offsetOfFirstByte is greater than file size,
+	    			// reset the first byte position to beginning of the file.
+	        		offsetOfFirstByte = 0L;    			
+	    			fileReadOffset=0;
+	    		}
+	    		
+	    		log.debug("Adaptor|" + adaptorID + "|seeking|" + fileReadOffset );
+	    		reader.seek(fileReadOffset);
+	    
+	    		long bufSize = len - fileReadOffset;
+	    		
+	    		if (conf == null)
+	    		{
+	    			ChukwaAgent agent = ChukwaAgent.getAgent();
+	    			if (agent != null)
+	    			{
+	    				conf = agent.getConfiguration();
+	        			if (conf != null)
+	        			{
+	        				MAX_READ_SIZE= conf.getInt("chukwaAgent.fileTailingAdaptor.maxReadSize", DEFAULT_MAX_READ_SIZE);
+	        				log.info("chukwaAgent.fileTailingAdaptor.maxReadSize: " + MAX_READ_SIZE);
+	        			}	
+	        			else
+	        			{
+	        				log.info("Conf is null, running in default mode");
+	        			}
+	    			}
+	    			else
+	    			{
+	    				log.info("Agent is null, running in default mode");
+	    			}
+	    		}
+	    		
+	    		if (bufSize > MAX_READ_SIZE) {
+	    			bufSize = MAX_READ_SIZE;
+	    			hasMoreData = true;
+	    		}
+	    		byte[] buf = new byte[(int) bufSize];
+	    		
+	    		
+	    		long curOffset = fileReadOffset;
+	    		
+	    		reader.read(buf);
+	    		assert reader.getFilePointer() == fileReadOffset + bufSize : " event size arithmetic is broken: "
+	    				+ " pointer is "
+	    				+ reader.getFilePointer()
+	    				+ " but offset is " + fileReadOffset + bufSize;
+	    
+	    		int bytesUsed = extractRecords(dest, fileReadOffset + offsetOfFirstByte, buf);
+	    		fileReadOffset = fileReadOffset + bytesUsed;
+	    		
+	    		
+	    		log.debug("Adaptor|" + adaptorID + "|start|" + curOffset + "|end|"+ fileReadOffset);
+	    		
+	    		
+	    	} else {
+	    		// file has rotated and no detection
+	    		reader.close();
+	    		reader=null;
+	    		fileReadOffset = 0L;
+	    		offsetOfFirstByte = 0L;
+	    		hasMoreData = true;
+				log.warn("Adaptor|" + adaptorID +"| file has rotated and no detection - reset counters to 0L");	    	
+	    	}
+	    } catch (IOException e) {
+	    	log.warn("failure reading " + toWatch, e);
+	    }
+	    attempts=0;
+	    return hasMoreData;
 	}
 	
   /**

+ 38 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TerminatorThread.java

@@ -0,0 +1,38 @@
+package org.apache.hadoop.chukwa.datacollection.adaptor.filetailer;
+
+import java.io.IOException;
+import java.util.TimerTask;
+
+import org.apache.hadoop.chukwa.datacollection.ChunkReceiver;
+import org.apache.hadoop.chukwa.datacollection.adaptor.AdaptorException;
+import org.apache.hadoop.chukwa.datacollection.adaptor.FileAdaptor;
+import org.apache.hadoop.chukwa.inputtools.plugin.metrics.ExecHelper;
+import org.apache.log4j.Logger;
+
+public class TerminatorThread extends Thread {
+	private static Logger log =Logger.getLogger(FileAdaptor.class);
+	private static FileTailingAdaptor adaptor = null;
+	private static ChunkReceiver eq = null;
+	
+	public TerminatorThread(FileTailingAdaptor adaptor, ChunkReceiver eq) {
+		this.adaptor = adaptor;
+		this.eq = eq;
+	}
+
+	public void run() {
+   	    log.info("Terminator thread started.");
+  	    try {
+  	    	while(adaptor.tailFile(eq)) {
+  	    		log.info("");
+  	    	}
+		} catch (InterruptedException e) {
+			log.info("Unable to send data to collector for 60 seconds, force shutdown.");
+		}
+        log.info("Terminator finished.");
+        try {
+        	adaptor.reader.close();
+        } catch (IOException ex) {
+        	
+        }
+	}
+}

+ 26 - 5
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/agent/AdaptorFactory.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.chukwa.datacollection.agent;
 
 import org.apache.hadoop.chukwa.datacollection.adaptor.Adaptor;
+import org.apache.log4j.Logger;
 
 /**
  * Produces new unconfigured adaptors, given the class name of the appender type
@@ -26,6 +27,7 @@ import org.apache.hadoop.chukwa.datacollection.adaptor.Adaptor;
  */
 public class AdaptorFactory {
    
+  static Logger log = Logger.getLogger(ChukwaAgent.class);
     /**
      * Instantiate an adaptor that can be added by the {@link ChukwaAgent}
      * @param className the name of the {@link Adaptor} class to instantiate
@@ -39,13 +41,32 @@ public class AdaptorFactory {
       obj = Class.forName(className).newInstance();
       if (Adaptor.class.isInstance(obj)){
         return (Adaptor) obj;
+      } 
+      else        
+        return null;
+    } catch (Exception e1){
+      log.warn("Error instantiating new adaptor by class name, " +
+      		"attempting again, but with default chukwa package prepended, i.e. " +
+      		"org.apache.hadoop.chukwa.datacollection.adaptor." + className + ". " 
+      		+ e1);
+      try{
+        //if failed, try adding default class prefix
+        Object obj2 = Class.forName(
+            "org.apache.hadoop.chukwa.datacollection.adaptor." +
+            className).newInstance();
+        if (Adaptor.class.isInstance(obj2)){
+          log.debug("Succeeded in finding class by adding default chukwa " +
+              "namespace prefix to class name profided");
+          return (Adaptor) obj2;
+        } 
+        else        
+          return null;
+      } catch (Exception e2) {
+        System.out.println("Also error instantiating new adaptor by classname" +
+        		"with prefix added" + e2);
+        return null;
       }
-      else return null;
-    } catch (Exception e){
-      System.out.println("Error instantiating new adaptor by class" + e);
-      return null;
     }
-    
   }
   
 }

+ 18 - 6
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/agent/AgentControlSocketListener.java

@@ -56,11 +56,14 @@ public class AgentControlSocketListener extends Thread {
       InputStream in = connection.getInputStream();
       BufferedReader br = new BufferedReader(new InputStreamReader(in));
       PrintStream out = new PrintStream(new BufferedOutputStream(connection.getOutputStream()));
+      //out.println("You are connected to the chukwa agent on "+ InetAddress.getLocalHost().getCanonicalHostName());
+      //out.flush();
       String cmd = null;
       while((cmd = br.readLine()) != null)  {
         processCommand(cmd, out);
       }
-      log.info("control connection closed");
+      if (log.isDebugEnabled())
+  		{ log.debug("control connection closed");}
       }
       catch(SocketException e ) {
         if(e.getMessage().equals("Socket Closed"))
@@ -78,7 +81,8 @@ public class AgentControlSocketListener extends Thread {
      */
     public void processCommand(String cmd, PrintStream out) throws IOException  {
       String[] words = cmd.split(" ");
-      log.info("command from " + connection.getRemoteSocketAddress() + ":"+ cmd);
+      if (log.isDebugEnabled())
+  		{ log.debug("command from " + connection.getRemoteSocketAddress() + ":"+ cmd);}
       
       if(words[0].equalsIgnoreCase("help"))  {
         out.println("you're talking to the Chukwa agent.  Commands available: ");
@@ -88,6 +92,7 @@ public class AgentControlSocketListener extends Thread {
         out.println("list -- list running adaptors");
         out.println("close -- close this connection");
         out.println("stopagent -- stop the whole agent process");
+        out.println("reloadCollectors -- reload the list of collectors");
         out.println("help -- print this message");
         out.println("\t Command names are case-blind.");
       }
@@ -123,9 +128,15 @@ public class AgentControlSocketListener extends Thread {
           out.println("OK adaptor "+ num+ " stopped");
         }
       }
-      else if(words[0].equalsIgnoreCase("list") )  {
+      else if(words[0].equalsIgnoreCase("reloadCollectors"))  {
+            agent.getConnector().reloadConfiguration();
+            out.println("OK reloadCollectors done");
+        }else if(words[0].equalsIgnoreCase("list") )  {
         Map<Long, Adaptor> adaptorsByNumber = agent.getAdaptorList();
-        System.out.println("number of adaptors: " + adaptorsByNumber.size());
+        
+        if (log.isDebugEnabled())
+    		{ log.debug("number of adaptors: " + adaptorsByNumber.size());}
+        
         synchronized(adaptorsByNumber)   {
           for(Map.Entry<Long, Adaptor> a: adaptorsByNumber.entrySet())  {
             try{
@@ -168,7 +179,7 @@ public class AgentControlSocketListener extends Thread {
   }
   
   public String formatAdaptorStatus(Adaptor a)  throws AdaptorException  {
-    return a.getClass().getCanonicalName() + " " + a.getCurrentStatus() + " " + agent.getOffset(a);
+    return a.getClass().getCanonicalName() + " " + a.getCurrentStatus();
   }
 
   /**
@@ -186,7 +197,8 @@ public class AgentControlSocketListener extends Thread {
     {
       try {
         Socket connection = s.accept();
-        log.info("new connection from " + connection.getInetAddress());
+        if (log.isDebugEnabled())
+        	{ log.debug("new connection from " + connection.getInetAddress());}
         ListenThread l = new ListenThread(connection);
         l.setDaemon(true);
         l.start();

+ 364 - 248
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/agent/ChukwaAgent.java

@@ -23,6 +23,7 @@ import org.apache.hadoop.chukwa.datacollection.adaptor.*;
 import org.apache.hadoop.chukwa.datacollection.connector.*;
 import org.apache.hadoop.chukwa.datacollection.connector.http.HttpConnector;
 import org.apache.hadoop.chukwa.datacollection.test.ConsoleOutConnector;
+import org.apache.hadoop.chukwa.util.PidFile;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
@@ -32,257 +33,331 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.io.*;
 
 /**
- * The local agent daemon that runs on each machine.
- * This class is designed to be embeddable, for use in testing.
+ * The local agent daemon that runs on each machine. This class is designed to
+ * be embeddable, for use in testing.
  * 
  */
 public class ChukwaAgent
 {
-  boolean DO_CHECKPOINT_RESTORE = false;
-  boolean WRITE_CHECKPOINTS = false;
- 
-  static String tags = "";
+  boolean DO_CHECKPOINT_RESTORE = true;
+  //boolean WRITE_CHECKPOINTS = true;
+
   static Logger log = Logger.getLogger(ChukwaAgent.class);
+  static ChukwaAgent agent = null;
+  private static PidFile pFile = null;
+
+  public static ChukwaAgent getAgent()
+  {
+    return agent;
+  }
 
-  //doesn't need an equals(), comparator, etc
-  private static class Offset {
-    public Offset(long l, long id)  {
+  Configuration conf = null;
+  Connector connector = null;
+
+  // doesn't need an equals(), comparator, etc
+  private static class Offset
+  {
+    public Offset(long l, long id)
+    {
       offset = l;
       this.id = id;
     }
+
     private volatile long id;
     private volatile long offset;
   }
 
-  public static class AlreadyRunningException extends Exception {
+  public static class AlreadyRunningException extends Exception
+  {
 
     private static final long serialVersionUID = 1L;
 
-    public AlreadyRunningException() {
+    public AlreadyRunningException()
+    {
       super("Agent already running; aborting");
     }
   }
-  
-  
+
   private final Map<Adaptor, Offset> adaptorPositions;
 
-  //basically only used by the control socket thread.
+  // basically only used by the control socket thread.
   private final Map<Long, Adaptor> adaptorsByNumber;
 
-  File checkpointDir;   //lock this object to indicate checkpoint in progress
-  File initialAdaptors;
-  String CHECKPOINT_BASE_NAME;  //base filename for checkpoint files
-  int CHECKPOINT_INTERVAL_MS ;  //min interval at which to write checkpoints
-
+  private File checkpointDir; // lock this object to indicate checkpoint in
+  // progress
+  private File initialAdaptors;
+  private String CHECKPOINT_BASE_NAME; // base filename for checkpoint files
+  private int CHECKPOINT_INTERVAL_MS; // min interval at which to write
+  // checkpoints
+  private static String tags = "";
 
   private Timer checkpointer;
-  private volatile boolean needNewCheckpoint = false; //set to true if any event has happened
-  //that should cause a new checkpoint to be written
-
-
-  private long lastAdaptorNumber= 0;   //ID number of the last adaptor to be started
-  private int checkpointNumber; //id number of next checkpoint.
-  //should be protected by grabbing lock on checkpointDir
+  private volatile boolean needNewCheckpoint = false; // set to true if any
+  // event has happened
+  // that should cause a new checkpoint to be written
 
+  private long lastAdaptorNumber = 0; // ID number of the last adaptor to be
+  // started
+  private int checkpointNumber; // id number of next checkpoint.
+  // should be protected by grabbing lock on checkpointDir
 
   private final AgentControlSocketListener controlSock;
 
   /**
    * @param args
-   * @throws AdaptorException 
+   * @throws AdaptorException
    */
-  public static void main(String[] args) throws AdaptorException {
+  public static void main(String[] args) throws AdaptorException
+  {
+
+    pFile = new PidFile("Agent");
+    Runtime.getRuntime().addShutdownHook(pFile);
 
-    try{
-      System.out.println("usage:  LocalAgent [-restore] [default collector URL]");
-      ChukwaAgent agent = new ChukwaAgent();
-      if(agent.anotherAgentIsRunning()) {
-        System.out.println("another agent is running (or port has been usurped).  Bailing out now");
+    try
+    {
+      if (args.length > 0 && args[0].equals("-help")) {
+        System.out.println("usage:  LocalAgent [-noCheckPoint]" +
+            "[default collector URL]");
+        System.exit(0);
+      }
+      ChukwaAgent localAgent = new ChukwaAgent();
+
+      if (agent.anotherAgentIsRunning())
+      {
+        System.out
+            .println("another agent is running (or port has been usurped). " +
+            		"Bailing out now");
+        System.exit(-1);
       }
-        
-      Connector connector;
 
-      int uriArgNumber= 0;
-      if(args.length > 0)  {
-        if(args[0].equals("-restore")) {
-          agent.DO_CHECKPOINT_RESTORE = true;
+      int uriArgNumber = 0;
+      if (args.length > 0)
+      {
+        if (args[0].equalsIgnoreCase("-noCheckPoint"))
+        {
+          agent.DO_CHECKPOINT_RESTORE = false;
           uriArgNumber = 1;
         }
-        if(args[uriArgNumber].equals("local"))
-          connector = new ConsoleOutConnector(agent);
+        if (args[uriArgNumber].equals("local"))
+          agent.connector = new ConsoleOutConnector(agent);
         else
         {
-          if(!args[uriArgNumber].contains("://"))
+          if (!args[uriArgNumber].contains("://"))
             args[uriArgNumber] = "http://" + args[uriArgNumber];
-          connector = new HttpConnector(agent, args[uriArgNumber]);
+          agent.connector = new HttpConnector(agent, args[uriArgNumber]);
         }
-      }
-      else
-        connector = new HttpConnector(agent);
+      } else
+        agent.connector = new HttpConnector(agent);
 
-      connector.start();
+      agent.connector.start();
 
       log.info("local agent started on port " + agent.getControlSock().portno);
 
-    }	catch(AlreadyRunningException e){
-      log.error("agent started already on this machine with same portno ; bailing out");
-      System.out.println("agent started already on this machine with same portno ; bailing out");
-      System.exit(0); //better safe than sorry
-    } catch(Exception e) 	{
+    } catch (AlreadyRunningException e)
+    {
+      log
+          .error("agent started already on this machine with same portno;" +
+          		" bailing out");
+      System.out
+          .println("agent started already on this machine with same portno;" +
+          		" bailing out");
+      System.exit(0); // better safe than sorry
+    } catch (Exception e)
+    {
       e.printStackTrace();
     }
   }
-  private boolean anotherAgentIsRunning() {
+
+  private boolean anotherAgentIsRunning()
+  {
     return !controlSock.isBound();
   }
+
   /**
    * @return the number of running adaptors inside this local agent
    */
-  public int adaptorCount() {
-    return adaptorPositions.size();
+  public int adaptorCount()
+  {
+    return adaptorsByNumber.size();
   }
 
   public ChukwaAgent() throws AlreadyRunningException
   {
+    ChukwaAgent.agent = this;
+
     readConfig();
 
-    //almost always just reading this; so use a ConcurrentHM.
-    //since we wrapped the offset, it's not a structural mod.
-    adaptorPositions= new ConcurrentHashMap<Adaptor, Offset>();
+    // almost always just reading this; so use a ConcurrentHM.
+    // since we wrapped the offset, it's not a structural mod.
+    adaptorPositions = new ConcurrentHashMap<Adaptor, Offset>();
     adaptorsByNumber = new HashMap<Long, Adaptor>();
-    checkpointNumber=0;
-    try{
-      if(DO_CHECKPOINT_RESTORE)
+    checkpointNumber = 0;
+    try
+    {
+      if (DO_CHECKPOINT_RESTORE)
         restoreFromCheckpoint();
-    } catch(IOException e)  {
+    } catch (IOException e)
+    {
       log.warn("failed to restart from checkpoint: ", e);
     }
-    
-    try {
-      if(initialAdaptors != null && initialAdaptors.exists())
+
+    try
+    {
+      if (initialAdaptors != null && initialAdaptors.exists())
         readAdaptorsFile(initialAdaptors);
-    } catch(IOException e) {
-      log.warn("couldn't read user-specified file "+ initialAdaptors.getAbsolutePath());
+    } catch (IOException e)
+    {
+      log.warn("couldn't read user-specified file "
+          + initialAdaptors.getAbsolutePath());
     }
-    
+
     controlSock = new AgentControlSocketListener(this);
-    try {
-      controlSock.tryToBind(); //do this synchronously; if it fails, we know another agent is running.
-      controlSock.start();  //this sets us up as a daemon
+    try
+    {
+      controlSock.tryToBind(); // do this synchronously; if it fails, we know
+      // another agent is running.
+      controlSock.start(); // this sets us up as a daemon
       log.info("control socket started on port " + controlSock.portno);
-      
-      if(CHECKPOINT_INTERVAL_MS > 0)  {
+
+      if (CHECKPOINT_INTERVAL_MS > 0)
+      {
         checkpointer = new Timer();
         checkpointer.schedule(new CheckpointTask(), 0, CHECKPOINT_INTERVAL_MS);
       }
-    } catch(IOException e) {
+    } catch (IOException e)
+    {
       log.info("failed to bind to socket; aborting agent launch", e);
       throw new AlreadyRunningException();
     }
 
-  
   }
 
-  //FIXME: should handle bad lines here
+  // FIXME: should handle bad lines here
   public long processCommand(String cmd)
   {
     String[] words = cmd.split(" ");
-    if(words[0].equalsIgnoreCase("add"))
+    if (words[0].equalsIgnoreCase("add"))
     {
-      //words should contain (space delimited):
-      //  0) command ("add")
-      //  1) AdaptorClassname
-      //  2) dataType (e.g. "hadoop_log")
-      //  3) params <optional> 
-      //           (e.g. for files, this is filename,
-      //            but can be arbitrarily many space
-      //            delimited agent specific params )
-      //  4) offset
+      // words should contain (space delimited):
+      // 0) command ("add")
+      // 1) AdaptorClassname
+      // 2) dataType (e.g. "hadoop_log")
+      // 3) params <optional>
+      // (e.g. for files, this is filename,
+      // but can be arbitrarily many space
+      // delimited agent specific params )
+      // 4) offset
 
       long offset;
-      try  {
-        offset = Long.parseLong(words[words.length-1]);
-      } catch(NumberFormatException e) {
+      try
+      {
+        offset = Long.parseLong(words[words.length - 1]);
+      } catch (NumberFormatException e)
+      {
         log.warn("malformed line " + cmd);
         return -1L;
       }
       String adaptorName = words[1];
 
       Adaptor adaptor = AdaptorFactory.createAdaptor(adaptorName);
-      if(adaptor == null) {
-        log.warn("don't recognize adaptor name " + adaptorName);
+      if (adaptor == null)
+      {
+        log.warn("Error creating adaptor from adaptor name " + adaptorName);
         return -1L;
       }
-      
 
       String dataType = words[2];
-      
+      String streamName = "";
       String params = "";
-      if(words.length > 4){ //no argument
-        int begParams = adaptorName.length()+dataType.length()+6;//length("ADD x type ") = length(x) + 5, i.e. count letters & spaces
-        params = cmd.substring(begParams, cmd.length() - words[words.length-1].length() -1);
+      if (words.length > 4)
+      { // no argument
+        int begParams = adaptorName.length() + dataType.length() + 6;
+        // length("ADD x type ") = length(x) + 5, i.e. count letters & spaces
+        params = cmd.substring(begParams, cmd.length()
+            - words[words.length - 1].length() - 1);
+        streamName = params.substring(params.indexOf(" ") + 1, params.length());
       }
       long adaptorID;
-      synchronized(adaptorsByNumber) {
-        adaptorID  = ++lastAdaptorNumber;
+      synchronized (adaptorsByNumber)
+      {
+        for (Map.Entry<Long, Adaptor> a : adaptorsByNumber.entrySet())
+        {
+          if (streamName.intern() == a.getValue().getStreamName().intern())
+          {
+            log.warn(params + " already exist, skipping.");
+            return -1;
+          }
+        }
+        adaptorID = ++lastAdaptorNumber;
         adaptorsByNumber.put(adaptorID, adaptor);
-        adaptorPositions.put(adaptor, new Offset(offset,adaptorID));
-      }
-      
-      try {
-        adaptor.start(dataType, params, offset, DataFactory.getInstance().getEventQueue());
-        log.info("started a new adaptor, id = " +adaptorID);
-        return adaptorID ;
-        
-      } catch(AdaptorException e) {
-        log.warn("failed to start adaptor", e);
-        //FIXME: don't we need to clean up the adaptor maps here?
+        adaptorPositions.put(adaptor, new Offset(offset, adaptorID));
+        try
+        {
+          adaptor.start(adaptorID, dataType, params, offset, DataFactory
+              .getInstance().getEventQueue());
+          log.info("started a new adaptor, id = " + adaptorID);
+          return adaptorID;
+
+        } catch (Exception e)
+        {
+          log.warn("failed to start adaptor", e);
+          // FIXME: don't we need to clean up the adaptor maps here?
+        }
       }
-    }
-    else
+    } else
       log.warn("only 'add' command supported in config files");
 
     return -1;
   }
 
   /**
-   *  Tries to restore from a checkpoint file in checkpointDir.
-   *  There should usually only be one checkpoint present --
-   *  two checkpoints present implies a crash during
-   *  writing the higher-numbered one.
-   *  As a result, this method chooses the lowest-numbered file present.
-   *  
-   *  Lines in the checkpoint file are processed one at a time with processCommand();
-   *   
+   * Tries to restore from a checkpoint file in checkpointDir. There should
+   * usually only be one checkpoint present -- two checkpoints present implies a
+   * crash during writing the higher-numbered one. As a result, this method
+   * chooses the lowest-numbered file present.
+   * 
+   * Lines in the checkpoint file are processed one at a time with
+   * processCommand();
+   * 
    * @return true if the restore succeeded
    * @throws IOException
-   */ 
+   */
   public boolean restoreFromCheckpoint() throws IOException
   {
-    synchronized(checkpointDir)
+    synchronized (checkpointDir)
     {
-      String[] checkpointNames =  checkpointDir.list(new FilenameFilter()
+      String[] checkpointNames = checkpointDir.list(new FilenameFilter()
       {
-        public boolean accept(File dir, String name)  {
+        public boolean accept(File dir, String name)
+        {
           return name.startsWith(CHECKPOINT_BASE_NAME);
-        } 
+        }
       });
-      if(checkpointNames.length == 0)
+      
+      if (checkpointNames == null) {
+        log.error("Unable to list directories in checkpoint dir");
+        return false;
+      }
+      if (checkpointNames.length == 0)
       {
-        log.info("No checkpoints found in "+ checkpointDir);
+        log.info("No checkpoints found in " + checkpointDir);
         return false;
       }
 
-      if(checkpointNames.length > 2)
-        log.warn("expected at most two checkpoint files in " + checkpointDir +  "; saw " + checkpointNames.length);
-      else if(checkpointNames.length == 0)
+      if (checkpointNames.length > 2)
+        log.warn("expected at most two checkpoint files in " + checkpointDir
+            + "; saw " + checkpointNames.length);
+      else if (checkpointNames.length == 0)
         return false;
 
-      String lowestName=null;
-      int lowestIndex=Integer.MAX_VALUE;
-      for(String n: checkpointNames) {
-        int index = Integer.parseInt(n.substring(CHECKPOINT_BASE_NAME.length()));
-        if(index < lowestIndex)  {
+      String lowestName = null;
+      int lowestIndex = Integer.MAX_VALUE;
+      for (String n : checkpointNames)
+      {
+        int index = Integer
+            .parseInt(n.substring(CHECKPOINT_BASE_NAME.length()));
+        if (index < lowestIndex)
+        {
           lowestName = n;
           lowestIndex = index;
         }
@@ -294,203 +369,244 @@ public class ChukwaAgent
     }
     return true;
   }
+
   private void readAdaptorsFile(File checkpoint) throws FileNotFoundException,
       IOException
   {
-    BufferedReader br = new BufferedReader( new InputStreamReader(new FileInputStream(checkpoint)));
-    String cmd=null;
-    while((cmd = br.readLine()) != null)
+    BufferedReader br = new BufferedReader(new InputStreamReader(
+        new FileInputStream(checkpoint)));
+    String cmd = null;
+    while ((cmd = br.readLine()) != null)
       processCommand(cmd);
     br.close();
   }
 
   /**
    * Called periodically to write checkpoints
+   * 
    * @throws IOException
    */
-  public void writeCheckpoint() throws IOException
-  { 
+  public void writeCheckpoint() throws IOException {
     needNewCheckpoint = false;
-    synchronized(checkpointDir) {
+    synchronized (checkpointDir) {
       log.info("writing checkpoint " + checkpointNumber);
 
-      FileOutputStream fos = new FileOutputStream(
-          new File(checkpointDir, CHECKPOINT_BASE_NAME + checkpointNumber));
-      PrintWriter out = new PrintWriter( new BufferedWriter(
+      FileOutputStream fos = new FileOutputStream(new File(checkpointDir,
+          CHECKPOINT_BASE_NAME + checkpointNumber));
+      PrintWriter out = new PrintWriter(new BufferedWriter(
           new OutputStreamWriter(fos)));
 
-      for(Map.Entry<Adaptor, Offset> stat: adaptorPositions.entrySet()) {
-        try{
+      for (Map.Entry<Adaptor, Offset> stat : adaptorPositions.entrySet()) {
+        try {
           Adaptor a = stat.getKey();
           out.print("ADD " + a.getClass().getCanonicalName());
-          out.print(" ");
-          out.print(a.getType());
-          out.print(" " + a.getCurrentStatus() + " ");
-          out.println(stat.getValue().offset);
-        }  catch(AdaptorException e)  {
+          out.println(" " + a.getCurrentStatus());
+        } catch (AdaptorException e) {
           e.printStackTrace();
-        }//don't try to recover from bad adaptor yet
+        }// don't try to recover from bad adaptor yet
       }
 
       out.close();
-      File lastCheckpoint =  new File(checkpointDir, CHECKPOINT_BASE_NAME + (checkpointNumber-1));
-      log.debug("hopefully removing old checkpoint file " + lastCheckpoint.getAbsolutePath());
+      File lastCheckpoint = new File(checkpointDir, CHECKPOINT_BASE_NAME
+          + (checkpointNumber - 1));
+      log.debug("hopefully removing old checkpoint file "
+          + lastCheckpoint.getAbsolutePath());
       lastCheckpoint.delete();
       checkpointNumber++;
     }
   }
 
-  public void reportCommit(Adaptor src, long uuid)
-  {
+  public void reportCommit(Adaptor src, long uuid) {
     needNewCheckpoint = true;
     Offset o = adaptorPositions.get(src);
-    if(o != null) {
-      synchronized(o) { //order writes to offset, in case commits are processed out of order
-        if( uuid > o.offset)
+    if (o != null)
+    {
+      synchronized (o)
+      { // order writes to offset, in case commits are processed out of order
+        if (uuid > o.offset)
           o.offset = uuid;
       }
-      
-      log.info("got commit up to " + uuid + " on " + src+ " = "+ o.id);
-    }
-    else {
-      log.warn("got commit up to " + uuid +  "  for adaptor " +src + 
-          " that doesn't appear to be running: " + adaptorsByNumber.size() + " total");
+
+      log.info("got commit up to " + uuid + " on " + src + " = " + o.id);
+    } else
+    {
+      log.warn("got commit up to " + uuid + "  for adaptor " + src
+          + " that doesn't appear to be running: " + adaptorsByNumber.size()
+          + " total");
     }
   }
 
-  class CheckpointTask extends TimerTask  {
-    public void run()  {
-      try{
-        if(needNewCheckpoint ) {
+  class CheckpointTask extends TimerTask {
+    public void run()
+    {
+      try
+      {
+        if (needNewCheckpoint)
+        {
           writeCheckpoint();
         }
-      } catch(IOException e)  {
+      } catch (IOException e)
+      {
         log.warn("failed to write checkpoint", e);
       }
     }
   }
-  
-//for use only by control socket.
-  Map<Long, Adaptor> getAdaptorList()  {
-    return adaptorsByNumber; 
+
+  // for use only by control socket.
+  public Map<Long, Adaptor> getAdaptorList() {
+    return adaptorsByNumber;
   }
+
   /**
-   * Stop the adaptor with given ID number.
-   * Takes a parameter to indicate whether the adaptor should
-   * force out all remaining data, or just exit abruptly.
+   * Stop the adaptor with given ID number. Takes a parameter to indicate
+   * whether the adaptor should force out all remaining data, or just exit
+   * abruptly.
    * 
-   * If the adaptor is written correctly, its offset won't change after returning
-   * from shutdown.
+   * If the adaptor is written correctly, its offset won't change after
+   * returning from shutdown.
    * 
-   * @param number the adaptor to stop
-   * @param gracefully if true, shutdown, if false, hardStop
+   * @param number
+   *          the adaptor to stop
+   * @param gracefully
+   *          if true, shutdown, if false, hardStop
    * @return the number of bytes synched at stop. -1 on error
    */
-  public long stopAdaptor(long number, boolean gracefully)  {
+  public long stopAdaptor(long number, boolean gracefully) {
     Adaptor toStop;
     long offset = -1;
-    
-      //at most one thread can get past this critical section with toStop != null
-      //so if multiple callers try to stop the same adaptor, all but one will fail
-    synchronized(adaptorsByNumber) {
+
+    // at most one thread can get past this critical section with toStop != null
+    // so if multiple callers try to stop the same adaptor, all but one will
+    // fail
+    synchronized (adaptorsByNumber) {
       toStop = adaptorsByNumber.remove(number);
     }
-    if(toStop == null) {
+    if (toStop == null) {
       log.warn("trying to stop adaptor " + number + " that isn't running");
       return offset;
     }
-    try {
-      if(gracefully ) {
-
-        long bytesSentByAdaptor = toStop.shutdown(); //this can block
-        long unstableBytes = bytesSentByAdaptor -adaptorPositions.get(toStop).offset;
-        while(unstableBytes > 0 ) {
-          log.info("waiting for adaptor " + number +  "  to terminate " +
-              unstableBytes + " bytes are still uncommitted");
-          Thread.sleep(2000);
-          unstableBytes = bytesSentByAdaptor -adaptorPositions.get(toStop).offset;
-        }
+    
+    try {    	      
+      if (gracefully) {
+   	    offset = toStop.shutdown(); 
       }
-      else
-        toStop.hardStop();
-      Offset off = adaptorPositions.remove(toStop);  //next checkpoint will have the remove
-      offset = off == null ? -1 : off.offset;
-      needNewCheckpoint = true;
-
-    } catch(AdaptorException e) {
+    } catch (AdaptorException e) {
       log.error("adaptor failed to stop cleanly", e);
-    } catch(InterruptedException e) {
-      log.error("can't wait for adaptor to finish writing", e);
+    } finally {
+    	  needNewCheckpoint = true;
     }
     return offset;
   }
-  
+
+  public Configuration getConfiguration() {
+    return conf;
+  }
+
+  public Connector getConnector() {
+    return connector;
+  }
+
   protected void readConfig() {
-    Configuration conf = new Configuration();
+    conf = new Configuration();
+
     String chukwaHome = System.getenv("CHUKWA_HOME");
-    if (chukwaHome == null){
+    if (chukwaHome == null) {
       chukwaHome = ".";
     }
-    if(!chukwaHome.endsWith("/"))
-      chukwaHome = chukwaHome + "/";
-    
-    conf.addResource(new Path("conf/chukwa-agent-conf.xml"));
-    CHECKPOINT_BASE_NAME = conf.get("chukwaAgent.checkpoint.name", "chukwa_checkpoint_");
-    checkpointDir= new File(conf.get("chukwaAgent.checkpoint.dir", chukwaHome+ "/var/"));
-    CHECKPOINT_INTERVAL_MS= conf.getInt("chukwaAgent.checkpoint.interval", 5000);
-    DO_CHECKPOINT_RESTORE = conf.getBoolean("chukwaAgent.checkpoint.enabled", false);
-    if(DO_CHECKPOINT_RESTORE) {
-      WRITE_CHECKPOINTS = true;
-      log.info("checkpoints are enabled, period is " + CHECKPOINT_INTERVAL_MS);
+
+    if (!chukwaHome.endsWith("/")) {
+      chukwaHome = chukwaHome + File.separator;
+    }
+    log.info("Config - System.getenv(\"CHUKWA_HOME\"): [" + chukwaHome + "]");
+
+    String chukwaConf = System.getProperty("CHUKWA_CONF_DIR");
+    if (chukwaConf == null) {
+      chukwaConf = chukwaHome + "conf" + File.separator;
+    }
+    if (!chukwaHome.endsWith("/")) {
+      chukwaHome = chukwaHome + File.separator;
+    }
+    if (!chukwaConf.endsWith("/")) {
+        chukwaConf = chukwaConf + File.separator;    	
+    }
+    log.info("Config - System.getenv(\"CHUKWA_HOME\"): [" + chukwaHome + "]");
+
+    conf.addResource(new Path(chukwaConf + "chukwa-agent-conf.xml"));
+    DO_CHECKPOINT_RESTORE = conf.getBoolean("chukwaAgent.checkpoint.enabled",
+        true);
+    CHECKPOINT_BASE_NAME = conf.get("chukwaAgent.checkpoint.name",
+        "chukwa_checkpoint_");
+    checkpointDir = new File(conf.get("chukwaAgent.checkpoint.dir", chukwaHome
+        + "/var/"));
+    CHECKPOINT_INTERVAL_MS = conf.getInt("chukwaAgent.checkpoint.interval",
+        5000);
+    if (!checkpointDir.exists())
+    {
+      checkpointDir.mkdirs();
     }
-  //  String initialAdaptorsStr = conf.get("initial_adaptors_file");
-    
     tags = conf.get("chukwaAgent.tags", "cluster=\"unknown\"");
-    
-    initialAdaptors = new File(chukwaHome + "conf/initial_adaptors");
+
+    log.info("Config - chukwaHome: [" + chukwaHome + "]");
+    log.info("Config - CHECKPOINT_BASE_NAME: [" + CHECKPOINT_BASE_NAME + "]");
+    log.info("Config - checkpointDir: [" + checkpointDir + "]");
+    log.info("Config - CHECKPOINT_INTERVAL_MS: [" + CHECKPOINT_INTERVAL_MS
+        + "]");
+    log.info("Config - DO_CHECKPOINT_RESTORE: [" + DO_CHECKPOINT_RESTORE + "]");
+    log.info("Config - tags: [" + tags + "]");
+
+    if (DO_CHECKPOINT_RESTORE) {
+      needNewCheckpoint = true;
+      log.info("checkpoints are enabled, period is " + CHECKPOINT_INTERVAL_MS);
+    }
+
+    initialAdaptors = new File(chukwaConf + "initial_adaptors");
   }
-  
+
   public void shutdown() {
     shutdown(false);
   }
 
   /**
-   * Triggers agent shutdown.
-   * For now, this method doesn't shut down adaptors explicitly.  It probably should.
+   * Triggers agent shutdown. For now, this method doesn't shut down adaptors
+   * explicitly. It probably should.
    */
   public void shutdown(boolean exit) {
-    if(checkpointer != null)  
+    if (checkpointer != null)
       checkpointer.cancel();
-    
-    controlSock.shutdown(); //make sure we don't get new requests
-
+    controlSock.shutdown(); // make sure we don't get new requests
     try {
-      if(WRITE_CHECKPOINTS)
-        writeCheckpoint(); //write a last checkpoint here, before stopping adaptors
-    } catch(IOException e) { 
+      if (needNewCheckpoint)
+        writeCheckpoint(); // write a last checkpoint here, before stopping
+      // adaptors
+    } catch (IOException e) {
     }
-    
-    synchronized(adaptorsByNumber) {   //shut down each adaptor
-      for(Adaptor a: adaptorsByNumber.values()) {
-        try{
+
+    synchronized (adaptorsByNumber) { 
+      // shut down each adaptor
+      for (Adaptor a : adaptorsByNumber.values()) {
+        try {
           a.hardStop();
-        }catch(AdaptorException e) {
-          log.warn("failed to cleanly stop " + a,e);
+        } catch (AdaptorException e)
+        {
+          log.warn("failed to cleanly stop " + a, e);
         }
       }
     }
-    
-    if(exit)
+    if (exit)
       System.exit(0);
   }
-/**
- *   Returns the last offset at which a given adaptor was checkpointed
- * @param a the adaptor in question
- * @return that adaptor's last-checkpointed offset
- */
+
+  /**
+   * Returns the last offset at which a given adaptor was checkpointed
+   * 
+   * @param a
+   *          the adaptor in question
+   * @return that adaptor's last-checkpointed offset
+   */
   public long getOffset(Adaptor a) {
     return adaptorPositions.get(a).offset;
   }
+
   /**
    * Returns the control socket for this agent.
    */
@@ -499,7 +615,7 @@ public class ChukwaAgent
   }
 
   public static String getTags() {
-	    return tags;
+    return tags;
   }
 
 }

+ 25 - 14
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/agent/MemLimitQueue.java

@@ -18,8 +18,9 @@
 
 package org.apache.hadoop.chukwa.datacollection.agent;
 
-import java.util.*;
-//import java.util.concurrent.*;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
 
 import org.apache.hadoop.chukwa.Chunk;
 import org.apache.hadoop.chukwa.datacollection.ChunkQueue;
@@ -33,13 +34,14 @@ import org.apache.log4j.Logger;
  */
 public class MemLimitQueue implements ChunkQueue
 {
-
 	static Logger log = Logger.getLogger(WaitingQueue.class);
 	
 	private Queue<Chunk> queue = new LinkedList<Chunk>();
 	private long dataSize = 0;
 	private final long MAX_MEM_USAGE;
 
+
+	
   public MemLimitQueue(int limit) {
     MAX_MEM_USAGE = limit;
   }
@@ -47,15 +49,21 @@ public class MemLimitQueue implements ChunkQueue
 	/**
 	 * @see org.apache.hadoop.chukwa.datacollection.ChunkQueue#add(org.apache.hadoop.chukwa.Chunk)
 	 */
-	public void add(Chunk event) throws InterruptedException
+	public void add(Chunk chunk) throws InterruptedException
 	{
-	  assert event != null: "can't enqueue null chunks";
+	  assert chunk != null: "can't enqueue null chunks";
     synchronized(this) {
-      while(event.getData().length  + dataSize > MAX_MEM_USAGE)
-        this.wait();
-      
-      dataSize += event.getData().length;
-      queue.add(event);
+      while(chunk.getData().length  + dataSize > MAX_MEM_USAGE)
+      {
+    	  try 
+    	  { 
+    		  this.wait();
+    		  log.info("MemLimitQueue is full [" + dataSize +"]");
+    	  }
+    	  catch(InterruptedException e) {}
+      }
+      dataSize += chunk.getData().length;
+      queue.add(chunk);
       this.notifyAll();
     }
 	 
@@ -64,7 +72,7 @@ public class MemLimitQueue implements ChunkQueue
 	/**
 	 * @see org.apache.hadoop.chukwa.datacollection.ChunkQueue#collect(java.util.List, int)
 	 */
-	public void collect(List<Chunk> events,int maxCount) throws InterruptedException
+	public void collect(List<Chunk> events,int maxSize) throws InterruptedException
 	{
 		synchronized(this) {
 		  //we can't just say queue.take() here, since we're holding a lock.
@@ -72,10 +80,13 @@ public class MemLimitQueue implements ChunkQueue
 		    this.wait();
 		  }
 		  
-		  int i = 0;
-		  while(!queue.isEmpty() && (i++ < maxCount)) { 
+		  
+		  int size = 0;
+		  while(!queue.isEmpty() && (size < maxSize)) { 
 		    Chunk e = this.queue.remove();
-		    dataSize -= e.getData().length;
+		    int chunkSize = e.getData().length;
+		    size += chunkSize;
+		    dataSize -= chunkSize;
 		    events.add(e);
 		  }
 		  this.notifyAll();

+ 46 - 22
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/collector/CollectorStub.java

@@ -22,53 +22,77 @@ import org.mortbay.jetty.*;
 import org.mortbay.jetty.nio.SelectChannelConnector;
 import org.mortbay.jetty.servlet.*;
 import org.apache.hadoop.chukwa.datacollection.collector.servlet.ServletCollector;
-import org.apache.hadoop.chukwa.datacollection.writer.ConsoleWriter;
+import org.apache.hadoop.chukwa.datacollection.writer.*;
+import org.apache.hadoop.chukwa.util.PidFile;
 import org.apache.hadoop.chukwa.conf.ChukwaConfiguration;
 
 public class CollectorStub {
   
-  
-  public static void main(String[] args)
-  {
-    
+  static int THREADS = 80;
+  private static PidFile pFile = null;
+  public static Server jettyServer = null;
+  public static void main(String[] args) {
+	
+	pFile=new PidFile("Collector");
+	Runtime.getRuntime().addShutdownHook(pFile); 	 	  
     try {
       System.out.println("usage:  CollectorStub [portno] [pretend]");
-      System.out.println("note: if no portno defined, defaults to value in chukwa-site.xml");
+      System.out.println("note: if no portno defined, " +
+      		"defaults to value in chukwa-site.xml");
  
       ChukwaConfiguration conf = new ChukwaConfiguration();
       int portNum = conf.getInt("chukwaCollector.http.port", 9999);
-
+      THREADS = conf.getInt("chukwaCollector.http.threads", 80);
+      
       if(args.length != 0)
         portNum = Integer.parseInt(args[0]);
+      
+        //pick a writer.
       if(args.length > 1) {
         if(args[1].equals("pretend"))
           ServletCollector.setWriter(new ConsoleWriter(true));
         else if(args[1].equals("pretend-quietly"))
           ServletCollector.setWriter(new ConsoleWriter(false));
+        else if(args[1].equals("-classname")) {
+          if(args.length < 3)
+            System.err.println("need to specify a writer class");
+          else {
+            Class<?> writerClass = Class.forName(args[2]);
+            if(writerClass != null &&
+                ChukwaWriter.class.isAssignableFrom(writerClass))
+              ServletCollector.setWriter(
+                  (ChukwaWriter) writerClass.newInstance());
+            else
+              System.err.println(args[2]+ "is not a ChukwaWriter");
+          }
+        }
         else
-          System.out.println("WARNING: don't know what to do with command line arg "+ args[1]);
+          System.out.println("WARNING: unknown command line arg "+ args[1]);
       }
       
+        //set up jetty connector
       SelectChannelConnector jettyConnector = new SelectChannelConnector();
-      jettyConnector.setLowResourcesConnections(20);
-      jettyConnector.setLowResourceMaxIdleTime(1000);
+      jettyConnector.setLowResourcesConnections(THREADS-10);
+      jettyConnector.setLowResourceMaxIdleTime(1500);
       jettyConnector.setPort(portNum);
-      Server server = new Server(portNum);
-      server.setConnectors(new Connector[]{ jettyConnector});
-      org.mortbay.thread.BoundedThreadPool pool = new  org.mortbay.thread.BoundedThreadPool();
-      pool.setMaxThreads(30);
-      server.setThreadPool(pool);
-      Context root = new Context(server,"/",Context.SESSIONS);
+        //set up jetty server
+      jettyServer = new Server(portNum);
+      
+      jettyServer.setConnectors(new Connector[]{ jettyConnector});
+      org.mortbay.thread.BoundedThreadPool pool = 
+        new org.mortbay.thread.BoundedThreadPool();
+      pool.setMaxThreads(THREADS);
+      jettyServer.setThreadPool(pool);
+        //and add the servlet to it
+      Context root = new Context(jettyServer,"/",Context.SESSIONS);
       root.addServlet(new ServletHolder(new ServletCollector()), "/*");
-      server.start();
-      server.setStopAtShutdown(false);
+      jettyServer.start();
+      jettyServer.setStopAtShutdown(false);
      
       System.out.println("started http collector on port number " + portNum);
 
-    }
-    catch(Exception e)
-    {
-      e.printStackTrace();
+    } catch(Exception e) {
+     e.printStackTrace();
       System.exit(0);
     }
 

+ 112 - 51
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/collector/servlet/ServletCollector.java

@@ -18,7 +18,13 @@
 
 package org.apache.hadoop.chukwa.datacollection.collector.servlet;
 
-import java.io.*;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Timer;
+import java.util.TimerTask;
 
 import javax.servlet.ServletConfig;
 import javax.servlet.ServletException;
@@ -27,26 +33,30 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.chukwa.Chunk;
 import org.apache.hadoop.chukwa.ChunkImpl;
 import org.apache.hadoop.chukwa.datacollection.writer.SeqFileWriter;
+import org.apache.hadoop.chukwa.datacollection.writer.WriterException;
 import org.apache.log4j.Logger;
 
 public class ServletCollector extends HttpServlet
 {
 
-  static final boolean FANCY_DIAGNOSTICS = true;
+  static final boolean FANCY_DIAGNOSTICS = false;
 	static org.apache.hadoop.chukwa.datacollection.writer.ChukwaWriter writer = null;
 	 
   private static final long serialVersionUID = 6286162898591407111L;
   Logger log = Logger.getRootLogger();//.getLogger(ServletCollector.class);
-	  
   
-	public static void setWriter(org.apache.hadoop.chukwa.datacollection.writer.ChukwaWriter w) throws IOException
+	public static void setWriter(org.apache.hadoop.chukwa.datacollection.writer.ChukwaWriter w) throws WriterException
 	{
 	  writer = w;
 	  w.init();
 	}
-  
+	static long statTime = 0L;
+	static int numberHTTPConnection = 0;
+	static int numberchunks = 0;
+	
 	public void init(ServletConfig servletConf) throws ServletException
 	{
 	  
@@ -56,6 +66,20 @@ public class ServletCollector extends HttpServlet
 			return;
 		}
 		
+		
+		Timer statTimer = new Timer();
+		statTimer.schedule(new TimerTask()
+		{
+			public void run() 
+			{
+				log.info("stats:ServletCollector,numberHTTPConnection:" + numberHTTPConnection
+						 + ",numberchunks:"+numberchunks);
+				statTime = System.currentTimeMillis();
+				numberHTTPConnection = 0;
+				numberchunks = 0;
+			}
+		}, (1000), (60*1000));
+		
 		try
 		{
 			// read the application->pipeline settings from a config file in the format:
@@ -81,56 +105,75 @@ public class ServletCollector extends HttpServlet
 			if (writer == null)
 				writer =  new SeqFileWriter();
 
-		} catch (IOException e) {
+		} catch (WriterException e) {
 			throw new ServletException("Problem init-ing servlet", e);
 		}		
 	}
 
 	protected void accept(HttpServletRequest req, HttpServletResponse resp)
-			throws ServletException
+	throws ServletException
 	{
-	  ServletDiagnostics diagnosticPage = new ServletDiagnostics();
+		numberHTTPConnection ++;
+		ServletDiagnostics diagnosticPage = new ServletDiagnostics();
+		final long currentTime = System.currentTimeMillis();
 		try {
-	    
-		  final long currentTime = System.currentTimeMillis();
-		  log.debug("new post from " + req.getRemoteHost() + " at " + currentTime);
+
+			log.debug("new post from " + req.getRemoteHost() + " at " + currentTime);
 			java.io.InputStream in = req.getInputStream();
-						
+
 			ServletOutputStream l_out = resp.getOutputStream();
 			final DataInputStream di = new DataInputStream(in);
 			final int numEvents = di.readInt();
-		  //	log.info("saw " + numEvents+ " in request");
+			//	log.info("saw " + numEvents+ " in request");
 
-      if(FANCY_DIAGNOSTICS)
-        diagnosticPage.sawPost(req.getRemoteHost(), numEvents, currentTime);
-			for (int i = 0; i < numEvents; i++){
-				// TODO: pass new data to all registered stream handler methods for this chunk's stream
+			if(FANCY_DIAGNOSTICS)
+			{ diagnosticPage.sawPost(req.getRemoteHost(), numEvents, currentTime); }
+
+			List<Chunk> events = new LinkedList<Chunk>();
+			ChunkImpl logEvent = null;
+			StringBuilder sb = new StringBuilder();
+
+			for (int i = 0; i < numEvents; i++)
+			{
+				// TODO: pass new data to all registered stream handler 
+			  //       methods for this chunk's stream
 				// TODO: should really have some dynamic assignment of events to writers
 
-	      ChunkImpl logEvent =  ChunkImpl.read(di);
-
-	      if(FANCY_DIAGNOSTICS)
-	        diagnosticPage.sawChunk(logEvent, i);
-	      
-				// write new data to data sync file
-				if(writer != null) {
-				  writer.add(logEvent);  //save() blocks until data is written
-				  //this is where we ACK this connection
-					l_out.print("ok:");
-					l_out.print(logEvent.getData().length);
-					l_out.print(" bytes ending at offset ");
-					l_out.println(logEvent.getSeqID()-1);
-				}
-				else
-					l_out.println("can't write: no writer");	
+				logEvent =  ChunkImpl.read(di);
+				sb.append("ok:");
+				sb.append(logEvent.getData().length);
+				sb.append(" bytes ending at offset ");
+				sb.append(logEvent.getSeqID()-1).append("\n");
+
+				events.add(logEvent);
+
+				if(FANCY_DIAGNOSTICS)
+				{ diagnosticPage.sawChunk(logEvent, i); }
 			}
 
-      if(FANCY_DIAGNOSTICS)
-        diagnosticPage.doneWithPost();
-	    resp.setStatus(200);
-			
-		} catch (IOException e) 	{
-			log.warn("IO error", e);
+			// write new data to data sync file
+			if(writer != null) 
+			{
+				writer.add(events);
+				numberchunks += events.size();
+				//this is where we ACK this connection
+				l_out.print(sb.toString());
+			}
+			else
+			{
+				l_out.println("can't write: no writer");
+			}
+
+
+			if(FANCY_DIAGNOSTICS)
+			{ diagnosticPage.doneWithPost(); }
+
+			resp.setStatus(200);
+
+		} 
+		catch(Throwable e) 
+		{
+			log.warn("Exception talking to " +req.getRemoteHost() + " at " + currentTime , e);
 			throw new ServletException(e);
 		}
 	}
@@ -147,16 +190,30 @@ public class ServletCollector extends HttpServlet
 	protected void doGet(HttpServletRequest req, HttpServletResponse resp)
 			throws ServletException, IOException
 	{
-	  PrintStream out = new PrintStream(resp.getOutputStream());
-    resp.setStatus(200);
-	  out.println("<html><body><h2>Chukwa servlet running</h2>");
-	  if(FANCY_DIAGNOSTICS)
-	    ServletDiagnostics.printPage(out);
-	  out.println("</body></html>");
-//		accept(req,resp);
+	
+		PrintStream out = new PrintStream(resp.getOutputStream());
+		resp.setStatus(200);
+		
+	  String pingAtt = req.getParameter("ping");
+	  if (pingAtt!=null)
+	  {
+		  out.println("Date:" + ServletCollector.statTime);
+		  out.println("Now:" + System.currentTimeMillis());
+		  out.println("numberHTTPConnection:" + ServletCollector.numberHTTPConnection);
+		  out.println("numberchunks:" + ServletCollector.numberchunks);
+	  }
+	  else
+	  {
+		  out.println("<html><body><h2>Chukwa servlet running</h2>");
+		  if(FANCY_DIAGNOSTICS)
+		    ServletDiagnostics.printPage(out);
+		  out.println("</body></html>");
+	  }
+    
+	  
 	}
 
-  @Override	
+    @Override	
 	public String getServletInfo()
 	{
 		return "Chukwa Servlet Collector";
@@ -165,10 +222,14 @@ public class ServletCollector extends HttpServlet
 	@Override
 	public void destroy()
 	{
-	  synchronized(writer)
-	  {
-	    writer.close();
-	  }
+	  try
+	{
+		writer.close();
+	} catch (WriterException e)
+	{
+		log.warn("Exception during close", e);
+		e.printStackTrace();
+	}
 	  super.destroy();
 	}
 }

+ 20 - 12
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/collector/servlet/ServletDiagnostics.java

@@ -31,6 +31,10 @@ public class ServletDiagnostics {
 
   static Logger log=  Logger.getLogger(ServletDiagnostics.class);
   
+
+  static int CHUNKS_TO_KEEP = 50;
+  static int CHUNKS_TO_DISPLAY = 50;
+  
   private static class PostStats { //statistics about a chunk
     public PostStats(String src, int count, long receivedTs)
     {
@@ -75,7 +79,6 @@ public class ServletDiagnostics {
 
   static LinkedList<PostStats> lastPosts;
   PostStats curPost;
-  static int CHUNKS_TO_KEEP = 300;
 
   
   public void sawPost(String source, int chunks, long receivedTs) {
@@ -96,28 +99,33 @@ public class ServletDiagnostics {
     long timeWindowOfSample = Long.MAX_VALUE;
     long now = System.currentTimeMillis();
 
-
     out.println("<ul>");
     
     synchronized(lastPosts) {
+      int toSkip = lastPosts.size() - CHUNKS_TO_DISPLAY; 
+      
       if(!lastPosts.isEmpty())
         timeWindowOfSample = now -  lastPosts.peek().receivedTs;
       
       for(PostStats stats: lastPosts) {
-        out.print("<li>");
-        
-        out.print(stats.dataSize + " bytes from " + stats.src + " at timestamp " + stats.receivedTs);
-        out.println(" which was " +  ((now - stats.receivedTs)/ 1000) + " seconds ago");
         Long oldBytes = bytesFromHost.get(stats.src);
         long newBytes = stats.dataSize;
         if(oldBytes != null)
           newBytes += oldBytes;
         bytesFromHost.put(stats.src, newBytes);
-        out.println("<ol>");
-        for(int i =0; i < stats.count; ++i)
-          out.println("<li> "+ stats.lengths[i] + " bytes of type " +
-              stats.types[i] + ".  Adaptor name ="+ stats.names[i] +" </li>");
-        out.println("</ol></li>");
+        
+        if( -- toSkip < 0) { //done skipping
+          out.print("<li>");
+          
+          out.print(stats.dataSize + " bytes from " + stats.src + " at timestamp " + stats.receivedTs);
+          out.println(" which was " +  ((now - stats.receivedTs)/ 1000) + " seconds ago");
+  
+          out.println("<ol>");
+          for(int i =0; i < stats.count; ++i)
+            out.println("<li> "+ stats.lengths[i] + " bytes of type " +
+                stats.types[i] + ".  Adaptor name ="+ stats.names[i] +" </li>");
+          out.println("</ol></li>");
+        }
       }
     }
     out.println("</ul>");
@@ -137,7 +145,7 @@ public class ServletDiagnostics {
   public void doneWithPost() {
     synchronized(lastPosts) {
       if(lastPosts.size() > CHUNKS_TO_KEEP)
-        lastPosts.remove();
+        lastPosts.removeFirst();
       lastPosts.add(curPost);
     }
   }

+ 2 - 1
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/connector/Connector.java

@@ -42,5 +42,6 @@ public interface Connector
 
 	
 	public void start();
-  public void shutdown();
+    public void shutdown();
+    public void reloadConfiguration();
 }

+ 98 - 56
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/connector/http/HttpConnector.java

@@ -56,15 +56,18 @@ public class HttpConnector implements Connector, Runnable  {
 
   static Timer statTimer = null;
   static volatile int chunkCount = 0;
-  static final int MAX_EVENTS_PER_POST = 1000;
-  static final int MIN_POST_INTERVAL= 4 * 1000;
+  static final int MAX_SIZE_PER_POST = 2*1024*1024;
+  static final int MIN_POST_INTERVAL= 5 * 1000;
   static ChunkQueue chunkQueue;
   
   ChukwaAgent agent;
   String argDestination = null;
   
   private boolean stopMe = false;
-
+  private boolean reloadConfiguration = false;
+  private Iterator<String> collectors = null;
+  protected ChukwaSender connectorClient = null;
+  
   static{
     statTimer = new Timer();
     chunkQueue = DataFactory.getInstance().getEventQueue();
@@ -97,65 +100,104 @@ public class HttpConnector implements Connector, Runnable  {
 	}
 	
 	public void run(){
-	 	log.info("HttpConnector started at time:" + System.currentTimeMillis());
+		log.info("HttpConnector started at time:" + System.currentTimeMillis());
+
+		Iterator<String> destinations = null;
+
+		// build a list of our destinations from collectors
+		try{
+			destinations = DataFactory.getInstance().getCollectorURLs();
+		} catch (IOException e){
+			log.error("Failed to retreive list of collectors from " +
+					"conf/collectors file", e);
+		}
+
+		connectorClient = new ChukwaHttpSender();
+
+		if (argDestination != null) 
+		{
+			ArrayList<String> tmp = new ArrayList<String>();
+			tmp.add(argDestination);
+			collectors = tmp.iterator();
+			connectorClient.setCollectors(collectors);
+			log.info("using collector specified at agent runtime: " + argDestination);
+		} 
+		else if (destinations != null && destinations.hasNext()) 
+		{
+			collectors = destinations;
+			connectorClient.setCollectors(destinations);
+			log.info("using collectors from collectors file");
+		} 
+		else {
+			log.error("No collectors specified, exiting (and taking agent with us).");
+			agent.shutdown(true);//error is unrecoverable, so stop hard.
+			return;
+		}
+
+		try {
+			long lastPost = System.currentTimeMillis();
+			while(!stopMe) {
+				List<Chunk> newQueue = new ArrayList<Chunk>();
+				try {
+					//get all ready chunks from the chunkQueue to be sent
+					chunkQueue.collect(newQueue,MAX_SIZE_PER_POST); //FIXME: should really do this by size
+
+				} catch(InterruptedException e) {
+					System.out.println("thread interrupted during addChunks(ChunkQueue)");
+					Thread.currentThread().interrupt();
+					break;
+				}
+				int toSend = newQueue.size();
+				List<ChukwaHttpSender.CommitListEntry> results = connectorClient.send(newQueue);
+				log.info("sent " +toSend + " chunks, got back " + results.size() + " acks");
+				//checkpoint the chunks which were committed
+				for(ChukwaHttpSender.CommitListEntry cle : results) {
+					agent.reportCommit(cle.adaptor, cle.uuid);
+					chunkCount++;
+				}
+
+				if (reloadConfiguration)
+				{
+					connectorClient.setCollectors(collectors);
+					log.info("Resetting colectors");
+					reloadConfiguration = false;
+				}
+
+				long now = System.currentTimeMillis();
+				if( now - lastPost < MIN_POST_INTERVAL )  
+					Thread.sleep(now - lastPost);  //wait for stuff to accumulate
+				lastPost = now;
+			} //end of try forever loop
+			log.info("received stop() command so exiting run() loop to shutdown connector");
+		} catch(OutOfMemoryError e) {
+			log.warn("Bailing out",e);
+			System.exit(-1);
+		} catch(InterruptedException e) {
+			//do nothing, let thread die.
+			log.warn("Bailing out",e);
+			System.exit(-1);
+		}catch(java.io.IOException e) {
+			log.error("connector failed; shutting down agent");
+			agent.shutdown(true);
+		}
+	}
 
-	 	Iterator<String> destinations = null;
-	  
+	@Override
+	public void reloadConfiguration()
+	{
+		reloadConfiguration = true;
+		Iterator<String> destinations = null;
+		  
 	 	// build a list of our destinations from collectors
 	 	try{
-	    destinations = DataFactory.getInstance().getCollectors();
+	    destinations = DataFactory.getInstance().getCollectorURLs();
 	  } catch (IOException e){
 	    log.error("Failed to retreive list of collectors from conf/collectors file", e);
 	  }
-	  
-    ChukwaSender connectorClient = new ChukwaHttpSender();
-	  if (argDestination != null) {
-
-	    ArrayList<String> tmp = new ArrayList<String>();
-	    tmp.add(argDestination);
-      connectorClient.setCollectors(tmp.iterator());
-      log.info("using collector specified at agent runtime: " + argDestination);
-    } else if (destinations != null && destinations.hasNext()) {
-      connectorClient.setCollectors(destinations);
-      log.info("using collectors from collectors file");
-	  } else {
-	    log.error("No collectors specified, exiting (and taking agent with us).");
-	    agent.shutdown(true);//error is unrecoverable, so stop hard.
-	    return;
+	  if (destinations != null && destinations.hasNext()) 
+	  {
+		  collectors = destinations;
 	  }
-	  
-	  try {
-	    long lastPost = System.currentTimeMillis();
-  	  while(!stopMe) {
-  	    List<Chunk> newQueue = new ArrayList<Chunk>();
-  	    try {
-  	      //get all ready chunks from the chunkQueue to be sent
-  	      chunkQueue.collect(newQueue,MAX_EVENTS_PER_POST); //FIXME: should really do this by size
-  	     
-  	    } catch(InterruptedException e) {
-  	      System.out.println("thread interrupted during addChunks(ChunkQueue)");
-  	      Thread.currentThread().interrupt();
-  	      break;
-  	    }
-  	    int toSend = newQueue.size();
-  	    List<ChukwaHttpSender.CommitListEntry> results = connectorClient.send(newQueue);
-  	    log.info("sent " +toSend + " chunks, got back " + results.size() + " acks");
-  	    //checkpoint the chunks which were committed
-  	    for(ChukwaHttpSender.CommitListEntry cle : results) {
-          agent.reportCommit(cle.adaptor, cle.uuid);
-          chunkCount++;
-        }
-  	    long now = System.currentTimeMillis();
-  	    if( now - lastPost < MIN_POST_INTERVAL )  
-  	      Thread.sleep(now - lastPost);  //wait for stuff to accumulate
-        lastPost = now;
-  	  } //end of try forever loop
-  	  log.info("received stop() command so exiting run() loop to shutdown connector");
-  	} catch(InterruptedException e) {
-	  //do nothing, let thread die.
-  	}catch(java.io.IOException e) {
-  	  log.error("connector failed; shutting down agent");
-  	  agent.shutdown(true);
-    }
+    
 	}
 }

+ 40 - 14
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/controller/ChukwaAgentController.java

@@ -19,11 +19,24 @@
 package org.apache.hadoop.chukwa.datacollection.controller;
 
 
-import java.net.*;
-import java.io.*;
-import java.util.*;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
 
 import org.apache.hadoop.chukwa.datacollection.agent.ChukwaAgent;
+import org.apache.log4j.Logger;
 
 /**
  * A convenience library for applications to communicate to the {@link ChukwaAgent}. Can be used
@@ -31,14 +44,16 @@ import org.apache.hadoop.chukwa.datacollection.agent.ChukwaAgent;
  * use for handling log rations.  
  */
 public class ChukwaAgentController {
-  
-  public class AddAdaptorTask extends TimerTask {
-    String adaptorName;
-    String type;
-    String params;
-    long offset;
-    long numRetries;
-    long retryInterval;
+	static Logger log = Logger.getLogger(ChukwaAgentController.class);
+  public class AddAdaptorTask extends TimerTask 
+  {
+	  	
+	    String adaptorName;
+	    String type;
+	    String params;
+	    long offset;
+	    long numRetries;
+	    long retryInterval;
     
     AddAdaptorTask(String adaptorName, String type, String params,
         long offset, long numRetries, long retryInterval){
@@ -50,8 +65,18 @@ public class ChukwaAgentController {
       this.retryInterval = retryInterval;
     }
     @Override
-    public void run() {
-      add(adaptorName, type, params, offset, numRetries, retryInterval);
+    public void run() 
+    {
+    	try
+    	{
+    		log.info("Trying to resend the add command [" + adaptorName + "][" + offset + "][" + params +"] [" + numRetries+"]");
+    		add(adaptorName, type, params, offset, numRetries, retryInterval);
+    	}
+    	catch(Exception e)
+    	{
+    		log.warn("Exception in AddAdaptorTask.run", e);
+    		e.printStackTrace();
+    	} 
     }
   }
 
@@ -148,7 +173,7 @@ public class ChukwaAgentController {
   Map<Long, ChukwaAgentController.Adaptor> pausedAdaptors;
   String hostname;
   int portno;
-  private Timer addFileTimer = new Timer();
+  
   
   public ChukwaAgentController(){
     portno = DEFAULT_PORT;
@@ -218,6 +243,7 @@ public class ChukwaAgentController {
          System.out.println("Scheduling a agent connection retry for adaptor add() in another " +
              retryInterval + " milliseconds, " + numRetries + " retries remaining");
          
+         Timer addFileTimer = new Timer();
          addFileTimer.schedule(new AddAdaptorTask(adaptorName, type, params, offset, numRetries-1, retryInterval), retryInterval);
        }
      }else{

+ 50 - 17
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/sender/ChukwaHttpSender.java

@@ -26,8 +26,10 @@ import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.OutputStream;
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.Iterator;
 import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.commons.httpclient.HttpClient;
 import org.apache.commons.httpclient.HttpException;
@@ -39,6 +41,7 @@ import org.apache.commons.httpclient.methods.PostMethod;
 import org.apache.commons.httpclient.methods.RequestEntity;
 import org.apache.commons.httpclient.params.HttpMethodParams;
 import org.apache.hadoop.chukwa.Chunk;
+import org.apache.hadoop.chukwa.datacollection.DataFactory;
 import org.apache.hadoop.chukwa.datacollection.adaptor.Adaptor;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.log4j.Logger;
@@ -47,22 +50,23 @@ import org.apache.log4j.Logger;
  * Encapsulates all of the http setup and connection details needed for
  * chunks to be delivered to a collector.
  * <p>
- * On error, tries the list of available collectors, pauses for a minute, and then repeats.
+ * On error, tries the list of available collectors, pauses for a minute, 
+ * and then repeats.
  * </p>
  * <p> Will wait forever for collectors to come up. </p>
  */
 public class ChukwaHttpSender implements ChukwaSender{
   static final int MAX_RETRIES_PER_COLLECTOR = 4; //fast retries, in http client
-  static final int SENDER_RETRIES = 3; 
+  static final int SENDER_RETRIES = 14440; 
   static final int WAIT_FOR_COLLECTOR_REBOOT = 20 * 1000; 
     //FIXME: this should really correspond to the timer in RetryListOfCollectors
-  
+  static final int BLACK_LIST_TIME = 300 * 1000;
   static Logger log = Logger.getLogger(ChukwaHttpSender.class);
   static HttpClient client = null;
   static MultiThreadedHttpConnectionManager connectionManager = null;
   static String currCollector = null;
 
-  
+  protected static ConcurrentHashMap<Long, String> blackList = null; 
   protected Iterator<String> collectors;
   
   static
@@ -115,11 +119,11 @@ public class ChukwaHttpSender implements ChukwaSender{
   }
 
   public ChukwaHttpSender(){
-    //setup default collector
     ArrayList<String> tmp = new ArrayList<String>();
     this.collectors = tmp.iterator();
-    currCollector = "http://localhost:8080";
-    log.info("added a single collector to collector list in ConnectorClient constructor, it's hasNext is now: " + collectors.hasNext());
+    ConcurrentHashMap<Long, String> tmpHash = new ConcurrentHashMap<Long, String>();
+    this.blackList = tmpHash;
+    log.info("setting collectors to an empty iterator");
 
   }
   
@@ -135,21 +139,22 @@ public class ChukwaHttpSender implements ChukwaSender{
    * @param collectors
    */
   public void setCollectors(Iterator<String> collectors){
-    this.collectors = collectors; 
-    //setup a new destination from our list of collectors if one hasn't been set up
+    this.collectors = collectors;
+    this.blackList.clear();
+    //setup a new destination from our list of collectors if one isn't set up
     if (currCollector == null){
       if (collectors.hasNext()){
         currCollector = collectors.next();
       }
       else
-        log.error("No collectors to try in send(), not even trying to do doPost()");
+        log.error("No collectors to try in setCollectors()");
     }
   }
   
   
   /**
-   * grab all of the chunks currently in the chunkQueue, stores a copy of them locally, calculates
-   * their size, sets them up 
+   * grab all of the chunks currently in the chunkQueue, stores a copy of them 
+   * locally, calculates their size, sets them up 
    * @return array of chunk id's which were ACKed by collector
    */
   public List<CommitListEntry> send(List<Chunk> toSend) throws InterruptedException, IOException{
@@ -183,14 +188,29 @@ public class ChukwaHttpSender implements ChukwaSender{
       //need to pick a destination here
       PostMethod method = new PostMethod();
       try   {
+    	if(blackList.size()!=0) {
+    		for(long time: blackList.keySet()) {
+    			long now = new Date().getTime();
+    			if(now-time > BLACK_LIST_TIME) {
+    	    		log.info(blackList.get(time)+" release from black list.");
+    				blackList.remove(time);
+    			} else if(currCollector.intern()==blackList.get(time)) {
+    				currCollector = collectors.next();
+    			}
+    		}
+    	}
         doPost(method, postData, currCollector);
-
+        // rotate POST to collectors do not work.  All agent and collectors end up spending time to create TCP connections
+        // but unable to send any data.
+        // currCollector = collectors.next();        
         retries = SENDER_RETRIES; //reset count on success
         //if no exception was thrown from doPost, ACK that these chunks were sent
         return commitResults;
       } catch (Throwable e) {
         log.error("Http post exception", e);
         log.info("Checking list of collectors to see if another collector has been specified for rollover");
+        blackList.put(new Date().getTime(), currCollector);
+        log.info("Black list collector: "+currCollector);
         if (collectors.hasNext()){
           currCollector = collectors.next();
           log.info("Found a new collector to roll over to, retrying HTTP Post to collector " + currCollector);
@@ -200,6 +220,9 @@ public class ChukwaHttpSender implements ChukwaSender{
                 " ms (" + retries + "retries left)");
             Thread.sleep(WAIT_FOR_COLLECTOR_REBOOT);
             retries --;
+            // shuffle the list of collectors if all of them are not available.
+            this.collectors = DataFactory.getInstance().getCollectorURLs();
+            this.blackList.clear();
           } else {
             log.error("No more collectors to try rolling over to; aborting");
             throw new IOException("no collectors");
@@ -211,6 +234,11 @@ public class ChukwaHttpSender implements ChukwaSender{
         method.releaseConnection();
       }
     } //end retry loop
+    if(currCollector==null) {
+    	// reset the collector list, if ran out of collector to try.
+        this.collectors = DataFactory.getInstance().getCollectorURLs();
+        this.blackList.clear();    	
+    }
     return new ArrayList<CommitListEntry>();
   }
   
@@ -230,25 +258,30 @@ public class ChukwaHttpSender implements ChukwaSender{
         return !(e instanceof java.net.ConnectException) && (exec < MAX_RETRIES_PER_COLLECTOR);
       }
     });
+    
+    pars.setParameter(HttpMethodParams.SO_TIMEOUT , new Integer(30000));
+    
+    
+    
     method.setParams(pars);
     method.setPath(dest);
     
      //send it across the network
     method.setRequestEntity(data);
     
-    log.info("HTTP post to " + dest+" length = "+ data.getContentLength());
+    log.info(">>>>>> HTTP post to " + dest+" length = "+ data.getContentLength());
     // Send POST request
     
-    client.setTimeout(8000);
+    //client.setTimeout(15*1000);
     int statusCode = client.executeMethod(method);
       
     if (statusCode != HttpStatus.SC_OK)  {
-      log.error("HTTP post response statusCode: " +statusCode + ", statusLine: " + method.getStatusLine());
+      log.error(">>>>>> HTTP post response statusCode: " +statusCode + ", statusLine: " + method.getStatusLine());
       //do something aggressive here
       throw new HttpException("got back a failure from server");
     }
     //implicitly "else"
-    log.info("got success back from the remote collector; response length "+ method.getResponseContentLength());
+    log.info(">>>>>> HTTP Got success back from the remote collector; response length "+ method.getResponseContentLength());
 
       //FIXME: should parse acks here
     InputStream rstream = null;

+ 23 - 6
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/sender/RetryListOfCollectors.java

@@ -22,6 +22,8 @@ import java.io.*;
 import java.net.URL;
 import java.util.*;
 
+import org.apache.hadoop.conf.Configuration;
+
 /***
  * An iterator returning a list of Collectors to try.
  * This class is nondeterministic, since it puts collectors back on the list after some period.
@@ -29,6 +31,7 @@ import java.util.*;
  * No node will be polled more than once per maxRetryRateMs milliseconds. hasNext() will continue return
  * true if you have not called it recently.
  *
+ *
  */
 public class RetryListOfCollectors implements Iterator<String> {
 
@@ -36,25 +39,39 @@ public class RetryListOfCollectors implements Iterator<String> {
   List<String> collectors;
   long lastLookAtFirstNode;
   int nextCollector=0;
+  private String portNo; 
+  Configuration conf;
   
-
   public RetryListOfCollectors(File collectorFile, int maxRetryRateMs) throws IOException {
     this.maxRetryRateMs = maxRetryRateMs;
     lastLookAtFirstNode = 0;
     collectors = new ArrayList<String>();
+    conf = new Configuration();
+    portNo = conf.get("chukwaCollector.http.port","8080");
     
     try{
       BufferedReader br  = new BufferedReader(new FileReader(collectorFile));
       String line;
       while((line = br.readLine()) != null) {
-        if(!line.contains("://")) //no protocol, assume http
-          collectors.add("http://"+line);
-        else
-          collectors.add(line);
+        if(!line.contains("://")) { 
+        	//no protocol, assume http
+        	if(line.matches(":\\d+")) {
+                collectors.add("http://" + line);
+        	} else {
+                collectors.add("http://" + line + ":" + portNo + "/");
+        	}
+        } else {
+        	if(line.matches(":\\d+")) {
+                collectors.add(line);
+        	} else {
+                collectors.add(line + ":" + portNo + "/");
+        	}
+        	collectors.add(line);
+        }
       }
       br.close();
     }catch(FileNotFoundException e){
-      System.err.println("Error in RetryListOfCollectors() opening file conf/connectors file from agent, double check that you have set the CHUKWA_HOME environment variable. Also, ensure file exists and is in classpath");
+      System.err.println("Error in RetryListOfCollectors() opening file: collectors, double check that you have set the CHUKWA_CONF_DIR environment variable. Also, ensure file exists and is in classpath");
     }catch(IOException e){
       System.err.println("I/O error in RetryListOfcollectors instantiation in readLine() from specified collectors file");
       throw e;

+ 6 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/test/ConsoleOutConnector.java

@@ -88,4 +88,10 @@ public class ConsoleOutConnector extends Thread implements Connector {
     this.interrupt();
   }
 
+@Override
+public void reloadConfiguration()
+{
+	System.out.println("reloadConfiguration");
+}
+
 }

+ 5 - 4
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/ChukwaWriter.java

@@ -18,14 +18,15 @@
 
 package org.apache.hadoop.chukwa.datacollection.writer;
 
-import java.io.IOException;
+import java.util.List;
 
 import org.apache.hadoop.chukwa.Chunk;
 
 public interface ChukwaWriter
 {
-	void init() throws IOException;
-	void add(Chunk data) throws IOException;
-	void close();
+	public void init() throws WriterException;
+	public void add(Chunk data) throws WriterException;
+	public void add(List<Chunk> chunks) throws WriterException;
+	public void close() throws WriterException;;
 
 }

+ 106 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/ClientAck.java

@@ -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.datacollection.writer;
+
+import org.apache.log4j.Logger;
+
+public class ClientAck
+{
+	static Logger log = Logger.getLogger(ClientAck.class);
+	
+	// TODO move all constant to config
+	
+	public static final int OK = 100;
+	public static final int KO = -100;
+	public static final int KO_LOCK = -200;
+	
+	private long ts = 0;
+	
+	private Object lock = new Object();
+	private int status = 0;
+	private Throwable exception = null;
+	private int waitTime = 6*1000;// 6 secs
+	private int timeOut = 15*1000;
+	
+	public ClientAck()
+	{
+		this.ts = System.currentTimeMillis() + timeOut;
+	}
+	
+  public int getTimeOut()
+  {
+    return timeOut;
+  }
+
+  public void wait4Ack()
+	{
+		synchronized(lock)
+		{
+//			log.info(">>>>>>>>>>>>>>>>>>>>>>>>> Client synch");
+			while (this.status == 0)
+			{
+//				log.info(">>>>>>>>>>>>>>>>>>>>>>>>> Client Before wait");
+				try { lock.wait(waitTime);}
+				catch(InterruptedException e)
+				{}
+				long now = System.currentTimeMillis();
+				if (now > ts)
+				{
+					this.status = KO_LOCK;
+					this.exception = new RuntimeException("More than maximum time lock [" + this.toString() +"]");
+				}
+			}
+//			log.info("[" + Thread.currentThread().getName() + "] >>>>>>>>>>>>>>>>> Client after wait status [" + status +  "] [" + this.toString() + "]");
+		}
+	}
+
+	public void releaseLock(int status, Throwable exception)
+	{
+		this.exception = exception;
+		this.status = status;
+		
+//		log.info("[" + Thread.currentThread().getName() + "] <<<<<<<<<<<<<<<<< Server synch [" + status + "] ----->>>> [" + this.toString() + "]");
+		synchronized(lock)
+		{		
+//			log.info("<<<<<<<<<<<<<<< Server before notify");
+			lock.notifyAll();
+		}
+//		log.info("<<<<<<<<<<<<<<< Server after notify");
+	}
+	
+	public int getStatus()
+	{
+		return status;
+	}
+
+	public void setStatus(int status)
+	{
+		this.status = status;
+	}
+
+	public Throwable getException()
+	{
+		return exception;
+	}
+
+	public void setException(Throwable exception)
+	{
+		this.exception = exception;
+	}
+}

+ 13 - 3
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/ConsoleWriter.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.chukwa.datacollection.writer;
 
-import java.io.IOException;
+import java.util.List;
 import java.util.Timer;
 import java.util.TimerTask;
 
@@ -59,14 +59,14 @@ public class ConsoleWriter implements ChukwaWriter {
     statTimer.cancel();
   }
 
-  public void init() throws IOException
+  public void init() throws WriterException
   {
      System.out.println("----  DUMMY HDFS WRITER IN USE ---");
 
      statTimer.schedule(new StatReportingTask(), 1000,10*1000);
   }
 
-  public void add(Chunk data) throws IOException
+  public void add(Chunk data) throws WriterException
   {
     int startOffset = 0;
 
@@ -87,4 +87,14 @@ public class ConsoleWriter implements ChukwaWriter {
     }
   }
 
+@Override
+public void add(List<Chunk> chunks) throws WriterException
+{
+	for(Chunk chunk: chunks)
+	{
+		add(chunk);
+	}
+	
+}
+
 }

+ 24 - 10
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/InMemoryWriter.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.chukwa.datacollection.writer;
 
 import java.io.*;
+import java.util.List;
 
 import org.apache.hadoop.chukwa.Chunk;
 import org.apache.hadoop.chukwa.ChunkImpl;
@@ -25,29 +26,44 @@ import org.apache.hadoop.chukwa.ChunkImpl;
 public class InMemoryWriter implements ChukwaWriter {
 
   ByteArrayOutputStream buf;
-  
+
   public void close() {
     buf.reset();
   }
 
-  public void init() throws IOException {
+  public void init() throws WriterException {
     buf = new ByteArrayOutputStream();
   }
 
-  public void add(Chunk data) throws IOException {
+  public void add(Chunk data) throws WriterException {
     DataOutputStream dos = new DataOutputStream(buf);
-    data.write(dos);
-    synchronized(this) {
+    try {
+      data.write(dos);
+    } catch (IOException e) {
+      e.printStackTrace();
+      throw new WriterException(e);
+    }
+    synchronized (this) {
       notify();
     }
   }
-  
+
+  @Override
+  public void add(List<Chunk> chunks) throws WriterException {
+    for (Chunk chunk : chunks) {
+      add(chunk);
+    }
+
+  }
+
   DataInputStream dis = null;
+
   /**
    * Try to read bytes, waiting up to ms
+   * 
    * @param bytes amount to try to read
-   * @param ms  time to wait
-   * @return a newly read-in chunk
+   * @param ms time to wait
+  * @return a newly read-in chunk
    * @throws IOException
    */
   public Chunk readOutChunk(int bytes, int ms) throws IOException {
@@ -63,13 +79,11 @@ public class InMemoryWriter implements ChukwaWriter {
       }
       if(dis == null)
        dis = new DataInputStream( new ByteArrayInputStream(buf.toByteArray()));
-      
       return ChunkImpl.read(dis);
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       return null;
     }
-    
   }
 
 }

+ 248 - 178
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/SeqFileWriter.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Calendar;
+import java.util.List;
 import java.util.Timer;
 import java.util.TimerTask;
 
@@ -42,9 +43,13 @@ import org.apache.log4j.Logger;
  */
 public class SeqFileWriter implements ChukwaWriter
 {
-	static Logger log = Logger.getLogger(SeqFileWriter.class);
 	public static final boolean ENABLE_ROTATION = true;
-
+	
+	static final int STAT_INTERVAL_SECONDS = 30;
+	static final Object lock = new Object();
+	
+    static Logger log = Logger.getLogger(SeqFileWriter.class);
+  
 	private FileSystem fs = null;
 	private ChukwaConfiguration conf = null;
 
@@ -55,217 +60,281 @@ public class SeqFileWriter implements ChukwaWriter
 	private String currentFileName = null;
 	private FSDataOutputStream currentOutputStr = null;
 	private static SequenceFile.Writer seqFileWriter = null;
-
+	
+	private static ClientAck clientAck = new ClientAck();
+	private static long nextRotate = 0;
+	private static int rotateInterval = 1000*60;
+	
+	private static Timer clientAckTimer = null;
+	
 	private Timer timer = null;
 
 	private Timer statTimer = null;
 	private volatile long dataSize = 0;
 
-	public SeqFileWriter() throws IOException
+	
+	private int initWriteChunkRetries = 10;
+	private int writeChunkRetries = initWriteChunkRetries;
+	
+	public SeqFileWriter() throws WriterException
 	{
 		conf = new ChukwaConfiguration(true);
 		init();
 	}
 
-	public void init() throws IOException
+	public void init() throws WriterException
 	{
 		outputDir = conf.get("chukwaCollector.outputDir", "/chukwa");
 
-		int rotateInterval = conf.getInt("chukwaCollector.rotateInterval",
+		rotateInterval = conf.getInt("chukwaCollector.rotateInterval",
 				1000 * 60 * 5);//defaults to 5 minutes
+		nextRotate = System.currentTimeMillis() + rotateInterval;
+		
+		initWriteChunkRetries = conf.getInt("chukwaCollector.writeChunkRetries", 10);
+		writeChunkRetries = initWriteChunkRetries;
+		
 		//check if they've told us the file system to use
-    String fsname = conf.get("writer.hdfs.filesystem");
-    if (fsname == null || fsname.equals("")){
-      //otherwise try to get the filesystem from hadoop
-      fsname = conf.get("fs.default.name");
-    }
+	    String fsname = conf.get("writer.hdfs.filesystem");
+	    if (fsname == null || fsname.equals(""))
+	    {
+	      //otherwise try to get the filesystem from hadoop
+	      fsname = conf.get("fs.default.name");
+	    }
 		
 
 		log.info("rotateInterval is " + rotateInterval);
-		log.info("ENABLE_ROTATION is " + ENABLE_ROTATION);
 		log.info("outputDir is " + outputDir);
 		log.info("fsname is " + fsname);
 		log.info("filesystem type from hadoop-default.xml is "
 				+ conf.get("fs.hdfs.impl"));
 
-		if (fsname == null)
-		{
+		if (fsname == null) {
 			log.error("no filesystem name");
-			throw new IOException("no filesystem");
-		}
-		try
-		{
+			throw new WriterException("no filesystem");
+		}	try {
 			fs = FileSystem.get(new URI(fsname), conf);
-			if (fs == null)
-			{
+			if (fs == null) {
 				log.error("can't connect to HDFS at " + fs.getUri());
 				return;
 			} else
 				log.info("filesystem is " + fs.getUri());
-		} catch (IOException e)
-		{
+		} catch (IOException e) {
 			log.error(
 							"can't connect to HDFS, trying default file system instead (likely to be local)",
 							e);
-			try
-			{
+			try	{
 				fs = FileSystem.get(conf);
-			} catch (IOException err)
-			{
+			} catch (IOException err) {
 				log.error("can't connect to default file system either", e);
 			}
-		} catch (URISyntaxException e)
-		{
+		} catch (URISyntaxException e) 	{
 			log.error("problem generating new URI from config setting");
 			return;
 		}
 
-		calendar.setTimeInMillis(System.currentTimeMillis());
-		int minutes = calendar.get(Calendar.MINUTE);
-		// number of minutes at current time
-
-		int dec = minutes / 10; // 'tens' digit of current time
-
-		int m = minutes - (dec * 10); // 'units' digit
-		if (m < 5)
-		{
-			m = 5 - m;
-		} else
-		{
-			m = 10 - m;
-		}
-
-		log.info("Current date [" + calendar.getTime().toString()
-				+ "] next schedule [" + m + "]");
+		// Setup everything by rotating
 		rotate();
-
-		timer = new Timer();
-
-		if (ENABLE_ROTATION)
+		 
+		clientAckTimer = new Timer();
+		clientAckTimer.schedule(new TimerTask()
 		{
-			log.info("sink rotation enabled, rotating every " + rotateInterval
-					+ " millis");
-			timer.schedule(new TimerTask()
+			public void run() 
 			{
-				public void run()
+				synchronized (lock) 
 				{
-					rotate();
+					ClientAck previous = clientAck ;
+					SeqFileWriter.clientAck = new ClientAck();
+					
+					try
+					{
+						// SeqFile is uncompressed for now
+						// So we can flush every xx secs
+						// But if we're using block Compression
+						// this is not true anymore
+						// because this will trigger
+						// the compression
+						if (currentOutputStr != null)
+						{
+							currentOutputStr.flush(); 
+						}
+						previous.releaseLock(ClientAck.OK, null);
+						long now = System.currentTimeMillis();
+						if (now >= nextRotate)
+						{
+							nextRotate = System.currentTimeMillis() + rotateInterval;
+							rotate();
+						}
+					}
+					catch(Throwable e)
+					{
+						previous.releaseLock(ClientAck.KO, e);
+						log.warn("Exception when flushing ", e);
+						e.printStackTrace();
+					}	
 				}
+			}
 
-			}, Math.min(rotateInterval, m * 60 * 1000), rotateInterval);
-
-			statTimer = new Timer();
-		} else
-			log.warn("sink rotation is OFF!!");
-
-		statTimer.schedule(new StatReportingTask(), 1000, 60 * 1000);
+		}, (5*1000), (5*1000));
+		
+		statTimer = new Timer();
+		statTimer.schedule(new StatReportingTask(), 1000, STAT_INTERVAL_SECONDS * 1000);
+		
+		
+		
 	}
 
 	private class StatReportingTask extends TimerTask
 	{
 		private long lastTs = System.currentTimeMillis();
-		private long lastDataSize = 0;
 
 		public void run()
 		{
-			long time = System.currentTimeMillis();
-			long interval = time - lastTs;
+			
+		  long time = System.currentTimeMillis();
+			long currentDs = dataSize;
+			dataSize = 0;
+			
+		  long interval = time - lastTs;
 			lastTs = time;
 
-			long ds = dataSize;
-			long dataRate = 1000 * (ds - lastDataSize) / interval; // kb/sec
-			lastDataSize = ds;
-
-			log.info("stat=datacollection.writer.hdfs|dataSize=" + dataSize);
-			log.info("stat=datacollection.writer.hdfs|dataRate=" + dataRate);
+			long dataRate = 1000 * currentDs / interval; // kb/sec
+			log.info("stat:datacollection.writer.hdfs dataSize=" + currentDs + " dataRate=" + dataRate);
 		}
 	};
 
+	
 	void rotate()
 	{
 		calendar.setTimeInMillis(System.currentTimeMillis());
 
 		log.info("start Date [" + calendar.getTime() + "]");
-		//granularity of rollover directory structure is hourly
-		String newDir = new java.text.SimpleDateFormat("yyyy_dd_HH")
-				.format(calendar.getTime());
-
 		log.info("Rotate from " + Thread.currentThread().getName());
 
-		Path newDirPath = new Path(outputDir + "/" + newDir);
-		log.info("Rotate directory[" + newDirPath.toString() + "]");
-		try
-		{
-			if (!fs.exists(newDirPath))
-			{
-				log.info("Create new directory:" + newDirPath.toString());
-				try
-				{
-					fs.mkdirs(newDirPath);
-				} catch (Exception e)
-				{
-					if (!fs.exists(newDirPath))
-					{
-						log.info("Failed to create new directory:"
-								+ newDirPath.toString() + "] ", e);
-					}
-				}
-			} else // use the existing directory, because we haven't hit a new hour yet
-			{
-				log.info("Rotate from [" + Thread.currentThread().getName()
-						+ "] directory (" + newDirPath + ") already exists.");
-
-			}
-	    String newName = new java.text.SimpleDateFormat("yyyy_dd_HH_mm_ss_SSS").format(calendar.getTime());
-	    newName += "_" + new java.rmi.server.UID().toString();
-	    newName = newName.replace("-", "");
-	    newName = newName.replace(":", "");
-	    newName = newName.replace(".", "");
+		String newName = new java.text.SimpleDateFormat("yyyyddHHmmssSSS").format(calendar.getTime());
+		newName += "_" + new java.rmi.server.UID().toString();
+		newName = newName.replace("-", "");
+		newName = newName.replace(":", "");
+		newName = newName.replace(".", "");
+		newName = outputDir + "/" + newName.trim();
 
-			newName = newDirPath + "/" + newName.trim();
 
-			Path newOutputPath = new Path(newName + ".chukwa");
-
-			FSDataOutputStream newOutputStr = fs.create(newOutputPath);
-			FSDataOutputStream previousOutputStr = null;
-			Path previousPath = null;
-			String previousFileName = null;
-
-			synchronized (this)
+		synchronized (lock) 
+		{
+			try
 			{
-				previousOutputStr = currentOutputStr;
-				previousPath = currentPath;
-				previousFileName = currentFileName;
+				FSDataOutputStream previousOutputStr = currentOutputStr;
+				Path previousPath = currentPath;
+				String previousFileName = currentFileName;
 
-				currentOutputStr = newOutputStr;
-				currentPath = newOutputPath;
-				currentFileName = newName;
-				if (previousOutputStr != null)
+				if (previousOutputStr != null) 	
 				{
 					previousOutputStr.close();
 					fs.rename(previousPath,
 							new Path(previousFileName + ".done"));
 				}
-
-				// Turn compression ON if the 5 mins archives are big
+				Path newOutputPath = new Path(newName + ".chukwa");			
+				FSDataOutputStream newOutputStr = fs.create(newOutputPath);
+				currentOutputStr = newOutputStr;
+				currentPath = newOutputPath;
+				currentFileName = newName;
+				// Uncompressed for now
 				seqFileWriter = SequenceFile.createWriter(conf, newOutputStr,
 						ChukwaArchiveKey.class, ChunkImpl.class,
 						SequenceFile.CompressionType.NONE, null);
 			}
-		} catch (IOException e)
-		{
-			log.error("failed to do rotate", e);
+			catch (IOException e)
+			{
+				log.fatal("IO Exception in rotate. Exiting!");
+				e.printStackTrace();
+				// TODO  
+				// As discussed for now:
+				// Everytime this happen in the past it was because HDFS was down, 
+				// so there's nothing we can do
+				// Shutting down the collector for now
+				// Watchdog will re-start it automatically
+				System.exit(-1);
+			}		
 		}
+
 		log.debug("finished rotate()");
 	}
 
-	public synchronized void add(Chunk chunk) throws IOException
+	// TODO merge the 2 add functions
+	@Override
+	public void add(List<Chunk> chunks) throws WriterException
 	{
-		if (chunk != null)
+		if (chunks != null) 	
 		{
-			try
+			try 
+			{
+				ChukwaArchiveKey archiveKey = new ChukwaArchiveKey();
+
+				// FIXME compute this once an hour
+				// 
+				synchronized (calendar)
+				{
+					calendar.setTimeInMillis(System.currentTimeMillis());
+					calendar.set(Calendar.MINUTE, 0);
+					calendar.set(Calendar.SECOND, 0);
+					calendar.set(Calendar.MILLISECOND, 0);
+
+					archiveKey.setTimePartition(calendar.getTimeInMillis());
+				}
+
+				ClientAck localClientAck = null;					
+				synchronized(lock)
+				{
+					localClientAck = SeqFileWriter.clientAck;
+					for (Chunk chunk : chunks)
+					{
+						archiveKey.setDataType(chunk.getDataType());
+						archiveKey.setStreamName(chunk.getTags() + "/" + chunk.getSource() + "/" + chunk.getStreamName());
+						archiveKey.setSeqId(chunk.getSeqID());
+
+						if (chunk != null) 	
+						{
+							seqFileWriter.append(archiveKey, chunk);
+							// compute size for stats
+							dataSize += chunk.getData().length;
+						}
+
+					}
+				}// End synchro
+
+				localClientAck.wait4Ack();
+				if (localClientAck.getStatus() != ClientAck.OK)
+				{
+					log.warn("Exception after notyfyAll on the lock - Thread:" + Thread.currentThread().getName(),localClientAck.getException());
+					throw new WriterException(localClientAck.getException());
+				}
+				else
+				{
+					// sucess
+					writeChunkRetries = initWriteChunkRetries;
+				}
+
+			}
+			catch (IOException e) 
 			{
-				assert chunk instanceof ChunkImpl : "bad input type";
+				writeChunkRetries --;
+				log.error("Could not save the chunk. ", e);
+
+				if (writeChunkRetries < 0)
+				{
+					log.fatal("Too many IOException when trying to write a chunk, Collector is going to exit!");
+					System.exit(-1);
+				}
+				throw new WriterException(e);
+			}
+		}
+
+	}
+	
+	public void add(Chunk chunk) throws WriterException
+	{
+	  
+		if (chunk != null) 	{
+			try {
 				ChukwaArchiveKey archiveKey = new ChukwaArchiveKey();
 
 				// FIXME compute this once an hour
@@ -280,71 +349,72 @@ public class SeqFileWriter implements ChukwaWriter
 				}
 
 				archiveKey.setDataType(chunk.getDataType());
-				archiveKey.setStreamName(chunk.getStreamName());
+				archiveKey.setStreamName(chunk.getTags() + "/" + chunk.getSource() + "/" + chunk.getStreamName());
 				archiveKey.setSeqId(chunk.getSeqID());
 
-				seqFileWriter.append(archiveKey, chunk);
-
-				dataSize += chunk.getData().length;
-				// currentOutput.sync(); //force file out to stable storage on
-				// the cluster.
-				// note that seqFileWriter.sync() would do something completely
-				// different
-			} catch (IOException e)
+				ClientAck localClientAck = null;
+				synchronized(lock)
+				{
+					localClientAck = SeqFileWriter.clientAck;
+					log.info("[" + Thread.currentThread().getName() + "] Client >>>>>>>>>>>> Current Ack object ===>>>>" + localClientAck.toString());
+					seqFileWriter.append(archiveKey, chunk);
+					
+					// compute size for stats
+					dataSize += chunk.getData().length;
+				}
+				localClientAck.wait4Ack();
+				
+				if (localClientAck.getStatus() != ClientAck.OK)
+				{
+					log.warn("Exception after notyfyAll on the lock - Thread:" + Thread.currentThread().getName(),localClientAck.getException());
+					throw new WriterException(localClientAck.getException());
+				}	
+				else
+				{
+					// sucess
+					writeChunkRetries = initWriteChunkRetries;
+				}
+			} 
+			catch (IOException e) 
 			{
-				log.error(e.getMessage());
-				rotate();
-				throw e;
+				writeChunkRetries --;
+				log.error("Could not save the chunk. ", e);
+	
+				if (writeChunkRetries < 0)
+				{
+					log.fatal("Too many IOException when trying to write a chunk, Collector is going to exit!");
+					System.exit(-1);
+				}
+				throw new WriterException(e);
 			}
 		}
 	}
 
 	public void close()
 	{
-		synchronized (this)
+		synchronized (lock)
 		{
-			try
-			{
-				this.currentOutputStr.close();
+		  if (timer != null)
+			  timer.cancel();
+		  if (statTimer != null)
+			  statTimer.cancel();
+		  if (clientAckTimer != null)
+			  clientAckTimer.cancel();
+			try {
+				
+				if (this.currentOutputStr != null)
+				{
+					this.currentOutputStr.close();
+				}
+					
+				clientAck.releaseLock(ClientAck.OK, null);
 				fs.rename(currentPath, new Path(currentFileName + ".done"));
-			} catch (IOException e)
+			} catch (IOException e) 
 			{
+				clientAck.releaseLock(ClientAck.OK, e);
 				log.error("failed to close and rename stream", e);
 			}
 		}
 	}
 
-	/*
-	 * public static class SeqFileKey implements
-	 * org.apache.hadoop.io.WritableComparable<SeqFileKey>{
-	 * 
-	 * public long seqID; public String streamName; public long
-	 * collectorTimestamp;
-	 * 
-	 * public SeqFileKey() {} // for use in deserializing
-	 * 
-	 * SeqFileKey(Chunk event) { seqID = event.getSeqID(); streamName =
-	 * event.getStreamName() + "_" + event.getSource(); collectorTimestamp =
-	 * System.currentTimeMillis(); }
-	 * 
-	 * public void readFields(DataInput in) throws IOException { seqID =
-	 * in.readLong(); streamName = in.readUTF(); collectorTimestamp =
-	 * in.readLong(); }
-	 * 
-	 * public void write(DataOutput out) throws IOException {
-	 * out.writeLong(seqID); out.writeUTF(streamName);
-	 * out.writeLong(collectorTimestamp); }
-	 * 
-	 * public int compareTo(SeqFileKey o) { int cmp =
-	 * streamName.compareTo(o.streamName); if(cmp == 0) { if(seqID < o.seqID)
-	 * return -1; else if (seqID == o.seqID) return 0; else return 1; } else
-	 * return cmp; }
-	 * 
-	 * public boolean equals(Object o) { return (o instanceof SeqFileKey) &&
-	 * (compareTo((SeqFileKey) o) == 0); }
-	 * 
-	 * public int hashCode() { return streamName.hashCode() ^ (int)(seqID >> 32) ^
-	 * (int) seqID; }
-	 *  }
-	 */
 }

+ 29 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/WriterException.java

@@ -0,0 +1,29 @@
+package org.apache.hadoop.chukwa.datacollection.writer;
+
+public class WriterException extends Exception
+{
+
+	/**
+	 * 
+	 */
+	private static final long serialVersionUID = -4207275200546397145L;
+
+	public WriterException()
+	{}
+
+	public WriterException(String message)
+	{
+		super(message);
+	}
+
+	public WriterException(Throwable cause)
+	{
+		super(cause);
+	}
+
+	public WriterException(String message, Throwable cause)
+	{
+		super(message, cause);
+	}
+
+}

+ 132 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveBuilder.java

@@ -0,0 +1,132 @@
+/*
+ * 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.archive;
+
+import org.apache.hadoop.chukwa.ChukwaArchiveKey;
+import org.apache.hadoop.chukwa.ChunkImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.mapred.lib.IdentityReducer;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Logger;
+
+public class ChukwaArchiveBuilder extends Configured implements Tool
+{
+	static Logger log = Logger.getLogger(ChukwaArchiveBuilder.class);
+
+	static int printUsage()
+	{
+		System.out
+				.println("ChuckwaArchiveBuilder <Stream/DataType/Daily/Hourly> <input> <output>");
+		ToolRunner.printGenericCommandUsage(System.out);
+		return -1;
+	}
+	
+	public int run(String[] args) throws Exception
+	{
+	
+
+		// Make sure there are exactly 3 parameters left.
+		if (args.length != 3)
+		{
+			System.out.println("ERROR: Wrong number of parameters: "
+					+ args.length + " instead of 3.");
+			return printUsage();
+		}
+	
+		JobConf jobConf = new JobConf(getConf(), ChukwaArchiveBuilder.class);
+		jobConf.addResource(new Path("conf/chukwa-demux-conf.xml"));
+		
+		jobConf.setInputFormat(SequenceFileInputFormat.class);
+		
+		jobConf.setMapperClass(IdentityMapper.class);
+		jobConf.setReducerClass(IdentityReducer.class);
+		
+		if (args[0].equalsIgnoreCase("Daily"))
+		{
+			jobConf.setPartitionerClass(ChukwaArchiveDailyPartitioner.class);
+			jobConf.setOutputFormat(ChukwaArchiveDailyOutputFormat.class);
+			jobConf.setJobName("Chukwa-DailyArchiveBuilder");
+		}
+		else if (args[0].equalsIgnoreCase("Hourly"))
+		{
+			jobConf.setJobName("Chukwa-HourlyArchiveBuilder");
+			jobConf.setPartitionerClass(ChukwaArchiveHourlyPartitioner.class);
+			jobConf.setOutputFormat(ChukwaArchiveHourlyOutputFormat.class);			
+		}
+		else if (args[0].equalsIgnoreCase("DataType"))
+    {
+      jobConf.setJobName("Chukwa-HourlyArchiveBuilder-DataType");
+      int reduceCount = jobConf.getInt("chukwaArchiveBuilder.reduceCount", 1);
+      log.info("Reduce Count:" + reduceCount);
+      jobConf.setNumReduceTasks(reduceCount);
+      
+      jobConf.setPartitionerClass(ChukwaArchiveDataTypePartitioner.class);
+      jobConf.setOutputFormat(ChukwaArchiveDataTypeOutputFormat.class);     
+    }
+		else if (args[0].equalsIgnoreCase("Stream"))
+    {
+      jobConf.setJobName("Chukwa-HourlyArchiveBuilder-Stream");
+      int reduceCount = jobConf.getInt("chukwaArchiveBuilder.reduceCount", 1);
+      log.info("Reduce Count:" + reduceCount);
+      jobConf.setNumReduceTasks(reduceCount);
+      
+      jobConf.setPartitionerClass(ChukwaArchiveStreamNamePartitioner.class);
+      jobConf.setOutputFormat(ChukwaArchiveStreamNameOutputFormat.class);     
+    }
+		else
+		{
+			System.out.println("ERROR: Wrong Time partionning: "
+					+ args[0] + " instead of [Stream/DataType/Hourly/Daily].");
+			return printUsage();
+		}
+
+    
+		jobConf.set("mapred.compress.map.output", "true");
+		jobConf.set("mapred.map.output.compression.codec", "org.apache.hadoop.io.compress.LzoCodec");
+		jobConf.set("mapred.output.compress", "true");
+		jobConf.set("mapred.output.compression.type", "BLOCK");
+	
+		
+		
+		jobConf.setOutputKeyClass(ChukwaArchiveKey.class);
+		jobConf.setOutputValueClass(ChunkImpl.class);
+	
+		FileInputFormat.setInputPaths(jobConf, args[1]);
+		FileOutputFormat.setOutputPath(jobConf, new Path(args[2]));
+
+		JobClient.runJob(jobConf);
+		return 0;
+	}
+
+	public static void main(String[] args) throws Exception
+	{
+		int res = ToolRunner.run(new Configuration(),
+				new ChukwaArchiveBuilder(), args);
+		System.exit(res);
+	}
+}

+ 2 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveDailyOutputFormat.java

@@ -31,6 +31,7 @@ public class ChukwaArchiveDailyOutputFormat extends MultipleSequenceFileOutputFo
 	static Logger log = Logger.getLogger(ChukwaArchiveDailyOutputFormat.class);
 	SimpleDateFormat sdf = new SimpleDateFormat("yyyy_MM_dd");
 	
+	
 	@Override
 	protected String generateFileNameForKeyValue(ChukwaArchiveKey key, ChunkImpl chunk,
 			String name)
@@ -38,6 +39,7 @@ public class ChukwaArchiveDailyOutputFormat extends MultipleSequenceFileOutputFo
 		
 		if (log.isDebugEnabled())
 			{log.debug("ChukwaArchiveOutputFormat.fileName: " + sdf.format(key.getTimePartition()));}
+		
 		return sdf.format(key.getTimePartition()) + ".arc";
 	}
 }

+ 45 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveDataTypeOutputFormat.java

@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.chukwa.extraction.archive;
+
+
+import java.text.SimpleDateFormat;
+
+import org.apache.hadoop.chukwa.ChukwaArchiveKey;
+import org.apache.hadoop.chukwa.ChunkImpl;
+import org.apache.hadoop.mapred.lib.MultipleSequenceFileOutputFormat;
+import org.apache.log4j.Logger;
+
+public class ChukwaArchiveDataTypeOutputFormat extends MultipleSequenceFileOutputFormat<ChukwaArchiveKey, ChunkImpl>
+{
+	static Logger log = Logger.getLogger(ChukwaArchiveDataTypeOutputFormat.class);
+	SimpleDateFormat sdf = new SimpleDateFormat("yyyy_MM_dd");
+	
+	
+	@Override
+	protected String generateFileNameForKeyValue(ChukwaArchiveKey key, ChunkImpl chunk,
+			String name)
+	{
+		
+		if (log.isDebugEnabled())
+			{log.debug("ChukwaArchiveOutputFormat.fileName: " + sdf.format(key.getTimePartition()));}
+		
+		return chunk.getDataType() + "_" +sdf.format(key.getTimePartition()) + ".arc";
+	}
+}

+ 42 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveDataTypePartitioner.java

@@ -0,0 +1,42 @@
+/*
+ * 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.archive;
+
+import java.text.SimpleDateFormat;
+
+import org.apache.hadoop.chukwa.ChukwaArchiveKey;
+import org.apache.hadoop.chukwa.ChunkImpl;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Partitioner;
+
+public class ChukwaArchiveDataTypePartitioner<K, V> 
+	implements Partitioner<ChukwaArchiveKey,ChunkImpl>
+{
+	SimpleDateFormat sdf = new SimpleDateFormat("yyyy_MM_dd");
+	
+	public void configure(JobConf arg0)
+	{}
+
+	public int getPartition(ChukwaArchiveKey key,ChunkImpl chunk, int numReduceTasks)
+	{
+		
+		 return ( (chunk.getDataType() + "_" +sdf.format(key.getTimePartition() )).hashCode() & Integer.MAX_VALUE) % numReduceTasks;
+	}
+
+}

+ 35 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveStreamNameOutputFormat.java

@@ -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.extraction.archive;
+
+
+import org.apache.hadoop.chukwa.ChukwaArchiveKey;
+import org.apache.hadoop.chukwa.ChunkImpl;
+import org.apache.hadoop.mapred.lib.MultipleSequenceFileOutputFormat;
+
+public class ChukwaArchiveStreamNameOutputFormat extends MultipleSequenceFileOutputFormat<ChukwaArchiveKey, ChunkImpl>
+{
+
+  @Override
+  protected String generateLeafFileName(String name)
+  {
+    return "chukwaArchive-" + super.generateLeafFileName(name);
+  }
+  
+}

+ 38 - 0
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/extraction/archive/ChukwaArchiveStreamNamePartitioner.java

@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.chukwa.extraction.archive;
+
+import org.apache.hadoop.chukwa.ChukwaArchiveKey;
+import org.apache.hadoop.chukwa.ChunkImpl;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Partitioner;
+
+public class ChukwaArchiveStreamNamePartitioner<K, V> 
+	implements Partitioner<ChukwaArchiveKey,ChunkImpl>
+{
+	public void configure(JobConf arg0)
+	{}
+
+	public int getPartition(ChukwaArchiveKey key,ChunkImpl chunk, int numReduceTasks)
+	{
+		
+		 return ( (chunk.getSource() + "/" + chunk.getStreamName()).hashCode() & Integer.MAX_VALUE) % numReduceTasks;
+	}
+
+}

Alguns arquivos não foram mostrados porque muitos arquivos mudaram nesse diff