chukwa-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From asrab...@apache.org
Subject svn commit: r784294 - in /hadoop/chukwa/trunk: ./ src/java/org/apache/hadoop/chukwa/datacollection/adaptor/ src/java/org/apache/hadoop/chukwa/datacollection/agent/ src/java/org/apache/hadoop/chukwa/datacollection/collector/servlet/ src/java/org/apache/...
Date Fri, 12 Jun 2009 22:11:02 GMT
Author: asrabkin
Date: Fri Jun 12 22:11:01 2009
New Revision: 784294

URL: http://svn.apache.org/viewvc?rev=784294&view=rev
Log:
CHUKWA-5. Adaptors have persistent names.

Modified:
    hadoop/chukwa/trunk/CHANGES.txt
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/AbstractAdaptor.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/Adaptor.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/AdaptorManager.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/AgentControlSocketListener.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/ChukwaAgent.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/collector/servlet/ServletCollector.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/controller/ChukwaAgentController.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/log4j/ChukwaDailyRollingFileAppender.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/log4j/TaskLogAppender.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/tools/backfilling/BackfillingLoader.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/util/ConstRateAdaptor.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/util/MaxRateSender.java
    hadoop/chukwa/trunk/src/java/org/apache/hadoop/mapred/ChukwaJobTrackerInstrumentation.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/database/TestDatabaseIostat.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/TestExecAdaptor.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestCharFileTailingAdaptorUTF8.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileExpirationPolicy.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileTailingAdaptorBigRecord.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileTailingAdaptors.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestLogRotate.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestRawAdaptor.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestStartAtOffset.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestAgent.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestAgentConfig.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestCmd.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/connector/ChunkCatcherConnector.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/controller/TestAgentClient.java
    hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/validationframework/ChukwaAgentToCollectorValidator.java

Modified: hadoop/chukwa/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/CHANGES.txt?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/CHANGES.txt (original)
+++ hadoop/chukwa/trunk/CHANGES.txt Fri Jun 12 22:11:01 2009
@@ -30,6 +30,8 @@
 
   IMPROVEMENTS
 
+    CHUKWA-5. Adaptors have durable names. (asrabkin)
+
     CHUKWA-280. Added end to end test to detect iostat overflow. (Eric Yang)
 
     CHUKWA-278. Improve post process manager and metric data loader to support data loading from pig aggregation. (Eric Yang)

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/AbstractAdaptor.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/AbstractAdaptor.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/AbstractAdaptor.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/AbstractAdaptor.java Fri Jun 12 22:11:01 2009
@@ -25,7 +25,7 @@
 
   protected String type;
   protected ChunkReceiver dest;
-  protected long adaptorID;
+  protected String adaptorID;
   protected AdaptorManager control;
 
   @Override
@@ -34,7 +34,7 @@
   }
 
   @Override
-  public final void start(long adaptorID, String type, String status, long offset,
+  public final void start(String adaptorID, String type, String status, long offset,
       ChunkReceiver dest, AdaptorManager c) throws AdaptorException {
     this.adaptorID = adaptorID;
     this.type = type;

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/Adaptor.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/Adaptor.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/Adaptor.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/adaptor/Adaptor.java Fri Jun 12 22:11:01 2009
@@ -47,7 +47,7 @@
    * @param offset the stream offset of the first byte sent by this adaptor
    * @throws AdaptorException
    */
-  public void start(long adaptorID, String type, String status, long offset,
+  public void start(String adaptorID, String type, String status, long offset,
       ChunkReceiver dest, AdaptorManager c) throws AdaptorException;
 
   /**

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/AdaptorManager.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/AdaptorManager.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/AdaptorManager.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/AdaptorManager.java Fri Jun 12 22:11:01 2009
@@ -31,10 +31,10 @@
 
   Configuration getConfiguration();
   int adaptorCount();
-  long stopAdaptor(long number, boolean gracefully);
-  Adaptor getAdaptor(long id);
-  long processAddCommand(String cmd);
-  Map<Long, String> getAdaptorList();
+  long stopAdaptor(String id, boolean gracefully);
+  Adaptor getAdaptor(String id);
+  String processAddCommand(String cmd);
+  Map<String, String> getAdaptorList();
 
   static AdaptorManager NULL = new AdaptorManager() {
 
@@ -44,12 +44,12 @@
     }
 
     @Override
-    public Adaptor getAdaptor(long id) {
+    public Adaptor getAdaptor(String id) {
       return null;
     }
 
     @Override
-    public Map<Long, String> getAdaptorList() {
+    public Map<String, String> getAdaptorList() {
       return Collections.emptyMap();
     }
 
@@ -59,12 +59,12 @@
     }
 
     @Override
-    public long processAddCommand(String cmd) {
-      return 0;
+    public String processAddCommand(String cmd) {
+      return "";
     }
 
     @Override
-    public long stopAdaptor(long number, boolean gracefully) {
+    public long stopAdaptor(String id, boolean gracefully) {
       return 0;
     }
   };

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/AgentControlSocketListener.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/AgentControlSocketListener.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/AgentControlSocketListener.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/AgentControlSocketListener.java Fri Jun 12 22:11:01 2009
@@ -121,8 +121,8 @@
       } else if (words[0].equalsIgnoreCase("close")) {
         connection.close();
       } else if (words[0].equalsIgnoreCase("add")) {
-        long newID = agent.processAddCommand(cmd);
-        if (newID != -1)
+        String newID = agent.processAddCommand(cmd);
+        if (newID != null)
           out.println("OK add completed; new ID is " + newID);
         else
           out.println("failed to start adaptor...check logs for details");
@@ -130,33 +130,31 @@
         if (words.length < 2) {
           out.println("need to specify an adaptor to shut down, by number");
         } else {
-          long num = Long.parseLong(words[1]);
-          long offset = agent.stopAdaptor(num, true);
+          long offset = agent.stopAdaptor(words[1], true);
           if (offset != -1)
-            out.println("OK adaptor " + num + " stopping gracefully at "
+            out.println("OK adaptor " + words[1] + " stopping gracefully at "
                 + offset);
           else
-            out.println("FAIL: perhaps adaptor " + num + " does not exist");
+            out.println("FAIL: perhaps adaptor " + words[1] + " does not exist");
         }
       } else if (words[0].equalsIgnoreCase("stop")) {
         if (words.length < 2) {
           out.println("need to specify an adaptor to shut down, by number");
         } else {
-          long num = Long.parseLong(words[1]);
-          agent.stopAdaptor(num, false);
-          out.println("OK adaptor " + num + " stopped");
+          agent.stopAdaptor(words[1], false);
+          out.println("OK adaptor " + words[1] + " stopped");
         }
       } else if (words[0].equalsIgnoreCase("reloadCollectors")) {
         agent.getConnector().reloadConfiguration();
         out.println("OK reloadCollectors done");
       } else if (words[0].equalsIgnoreCase("list")) {
-        java.util.Map<Long, String> adaptorList = agent.getAdaptorList();
+        java.util.Map<String, String> adaptorList = agent.getAdaptorList();
 
         if (log.isDebugEnabled()) {
           log.debug("number of adaptors: " + adaptorList.size());
         }
 
-        for (Map.Entry<Long, String> a: adaptorList.entrySet()) {
+        for (Map.Entry<String, String> a: adaptorList.entrySet()) {
             out.print(a.getKey());
             out.print(") ");
             out.print(" ");

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/ChukwaAgent.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/ChukwaAgent.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/ChukwaAgent.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/agent/ChukwaAgent.java Fri Jun 12 22:11:01 2009
@@ -30,6 +30,8 @@
 import java.io.InputStreamReader;
 import java.io.OutputStreamWriter;
 import java.io.PrintWriter;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.regex.Matcher;
@@ -68,12 +70,12 @@
 
   // doesn't need an equals(), comparator, etc
   private static class Offset {
-    public Offset(long l, long id) {
+    public Offset(long l, String id) {
       offset = l;
       this.id = id;
     }
 
-    private volatile long id;
+    private final String id;
     private volatile long offset;
   }
 
@@ -90,7 +92,7 @@
 
   // basically only used by the control socket thread.
   //must be locked before access
-  private final Map<Long, Adaptor> adaptorsByNumber;
+  private final Map<String, Adaptor> adaptorsByName;
 
   private File checkpointDir; // lock this object to indicate checkpoint in
   // progress
@@ -102,9 +104,6 @@
   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
 
@@ -175,8 +174,8 @@
    */
   @Override
   public int adaptorCount() {
-    synchronized(adaptorsByNumber) {
-      return adaptorsByNumber.size();
+    synchronized(adaptorsByName) {
+      return adaptorsByName.size();
     }
   }
 
@@ -191,7 +190,7 @@
     // 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>();
+    adaptorsByName = new HashMap<String, Adaptor>();
     checkpointNumber = 0;
 
     boolean DO_CHECKPOINT_RESTORE = conf.getBoolean(
@@ -267,18 +266,23 @@
 
   // words should contain (space delimited):
   // 0) command ("add")
-  // 1) AdaptorClassname
-  // 2) dataType (e.g. "hadoop_log")
-  // 3) params <optional>
+  // 1) Optional adaptor name, followed by =
+  // 2) AdaptorClassname
+  // 3) dataType (e.g. "hadoop_log")
+  // 4) params <optional>
   // (e.g. for files, this is filename,
   // but can be arbitrarily many space
   // delimited agent specific params )
-  // 4) offset
+  // 5) offset
   private Pattern addCmdPattern = Pattern.compile("[aA][dD][dD]\\s+" // command "add",
                                                              // any case, plus
                                                              // at least one
                                                              // space
-      + "(\\S+)\\s+" // the adaptor classname, plus at least one space
+      + "(?:"   //noncapturing group
+      +	"([^\\s=]+)" //containing a string (captured) 
+      + "\\s*=\\s*" //then an equals sign, potentially set off with whitespace
+      + ")?" //end optional noncapturing group 
+      + "([^\\s=]+)\\s+" // the adaptor classname, plus at least one space. No '=' in name
       + "(\\S+)\\s+" // datatype, plus at least one space
       + "(?:" // start a non-capturing group, for the parameters
       + "(.*?)\\s+" // capture the actual parameters reluctantly, followed by
@@ -293,46 +297,52 @@
    * initial_adaptors and checkpoint files. So it makes sense to handle them here.
    * 
    */
-  public long processAddCommand(String cmd) {
+  public String processAddCommand(String cmd) {
     Matcher m = addCmdPattern.matcher(cmd);
     if (m.matches()) {
       long offset; // check for obvious errors first
       try {
-        offset = Long.parseLong(m.group(4));
+        offset = Long.parseLong(m.group(5));
       } catch (NumberFormatException e) {
         log.warn("malformed line " + cmd);
-        return -1L;
+        return null;
       }
 
-      String adaptorName = m.group(1);
-      String dataType = m.group(2);
-      String params = m.group(3);
+       
+      String adaptorID = m.group(1);
+      String adaptorClassName = m.group(2);
+      String dataType = m.group(3);
+      String params = m.group(4);
       if (params == null)
         params = "";
 
-      Adaptor adaptor = AdaptorFactory.createAdaptor(adaptorName);
+      if(adaptorID == null)
+        adaptorID = synthesizeAdaptorID(adaptorClassName, dataType, params);
+      
+      Adaptor adaptor = AdaptorFactory.createAdaptor(adaptorClassName);
       if (adaptor == null) {
-        log.warn("Error creating adaptor from adaptor name " + adaptorName);
-        return -1L;
+        log.warn("Error creating adaptor of class " + adaptorClassName);
+        return null;
       }
 
-      long adaptorID;
-      synchronized (adaptorsByNumber) {
-        for (Map.Entry<Long, Adaptor> a : adaptorsByNumber.entrySet()) {
+      synchronized (adaptorsByName) {
+        
+        /*for (Map.Entry<Long, Adaptor> a : adaptorsByName.entrySet()) {
           if (params.indexOf(a.getValue().getStreamName())!=-1) {
             log.warn(params + " already exist, skipping.");
-            return -1;
+            return null;
           }
-        }
-        adaptorID = ++lastAdaptorNumber;
-        adaptorsByNumber.put(adaptorID, adaptor);
+        }*/
+        if(adaptorsByName.containsKey(adaptorID))
+          return adaptorID;
+        adaptorsByName.put(adaptorID, adaptor);
         adaptorPositions.put(adaptor, new Offset(offset, adaptorID));
         needNewCheckpoint = true;
         try {
           adaptor.start(adaptorID, dataType, params, offset, DataFactory
               .getInstance().getEventQueue(), this);
           log.info("started a new adaptor, id = " + adaptorID);
-          ChukwaAgent.agentMetrics.adaptorCount.set(adaptorsByNumber.size());
+          ChukwaAgent.agentMetrics.adaptorCount.set(adaptorsByName.size());
           ChukwaAgent.agentMetrics.addedAdaptor.inc();
           return adaptorID;
 
@@ -345,7 +355,31 @@
       log.warn("only 'add' command supported in config files; cmd was: " + cmd);
     // no warning for blank line
 
-    return -1;
+    return null;
+  }
+
+  private String synthesizeAdaptorID(String adaptorClassName, String dataType,
+      String params) {
+    MessageDigest md;
+    try {
+      md = MessageDigest.getInstance("MD5");
+
+      md.update(adaptorClassName.getBytes());
+      md.update(dataType.getBytes());
+      md.update(params.getBytes());
+      StringBuilder sb = new StringBuilder();
+      byte[] bytes = md.digest();
+      for(int i=0; i < bytes.length; ++i) {
+        if( (bytes[i] & 0xF0) == 0)
+          sb.append('0');
+        sb.append( Integer.toHexString(0xFF & bytes[i]) );
+      }
+      return sb.toString();
+    } catch (NoSuchAlgorithmException e) {
+      log.fatal("MD5 apparently doesn't work on your machine; bailing", e);
+      shutdown(true);//abort agent
+    }
+    return null;
   }
 
   /**
@@ -426,8 +460,8 @@
       PrintWriter out = new PrintWriter(new BufferedWriter(
           new OutputStreamWriter(fos)));
 
-      for (Map.Entry<Long, String> stat : getAdaptorList().entrySet()) {
-          out.print("ADD " + stat.getValue());
+      for (Map.Entry<String, String> stat : getAdaptorList().entrySet()) {
+          out.println("ADD "+ stat.getKey()+ " = " + stat.getValue());
       }
 
       out.close();
@@ -480,10 +514,10 @@
  * adaptor status strings.
  * @return
  */
-  public Map<Long, String> getAdaptorList() {
-    Map<Long, String> adaptors = new HashMap<Long, String>();
-    synchronized (adaptorsByNumber) {
-      for (Map.Entry<Long, Adaptor> a : adaptorsByNumber.entrySet()) {
+  public Map<String, String> getAdaptorList() {
+    Map<String, String> adaptors = new HashMap<String, String>();
+    synchronized (adaptorsByName) {
+      for (Map.Entry<String, Adaptor> a : adaptorsByName.entrySet()) {
         try {
           adaptors.put(a.getKey(), formatAdaptorStatus(a.getValue()));
         } catch (AdaptorException e) {
@@ -506,33 +540,33 @@
    * @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(String name, 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) {
-      toStop = adaptorsByNumber.remove(number);
+    synchronized (adaptorsByName) {
+      toStop = adaptorsByName.remove(name);
     }
     if (toStop == null) {
-      log.warn("trying to stop adaptor " + number + " that isn't running");
+      log.warn("trying to stop adaptor " + name + " that isn't running");
       return offset;
     } else {
       adaptorPositions.remove(toStop);
     }
-    ChukwaAgent.agentMetrics.adaptorCount.set(adaptorsByNumber.size());
+    ChukwaAgent.agentMetrics.adaptorCount.set(adaptorsByName.size());
     ChukwaAgent.agentMetrics.removedAdaptor.inc();
     
     try {
       if (gracefully) {
         offset = toStop.shutdown();
-        log.info("shutdown on adaptor: " + number + ", "
+        log.info("shutdown on adaptor: " + name + ", "
             + toStop.getCurrentStatus());
       } else {
         toStop.hardStop();
-        log.info("hardStop on adaptorId: " + number + ", "
+        log.info("hardStop on adaptorId: " + name + ", "
             + toStop.getCurrentStatus());
       }
     } catch (AdaptorException e) {
@@ -549,9 +583,9 @@
   }
   
   @Override
-  public Adaptor getAdaptor(long l) {
-    synchronized(adaptorsByNumber) {
-      return adaptorsByNumber.get(l);
+  public Adaptor getAdaptor(String name) {
+    synchronized(adaptorsByName) {
+      return adaptorsByName.get(name);
     }
   }
 
@@ -608,9 +642,9 @@
     }
     // adaptors
 
-    synchronized (adaptorsByNumber) {
+    synchronized (adaptorsByName) {
       // shut down each adaptor
-      for (Adaptor a : adaptorsByNumber.values()) {
+      for (Adaptor a : adaptorsByName.values()) {
         try {
           a.hardStop();
         } catch (AdaptorException e) {
@@ -618,7 +652,7 @@
         }
       }
     }
-    adaptorsByNumber.clear();
+    adaptorsByName.clear();
     adaptorPositions.clear();
     if (exit)
       System.exit(0);

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/collector/servlet/ServletCollector.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/collector/servlet/ServletCollector.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/collector/servlet/ServletCollector.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/collector/servlet/ServletCollector.java Fri Jun 12 22:11:01 2009
@@ -120,7 +120,7 @@
     final long currentTime = System.currentTimeMillis();
     try {
 
-      log.debug("new post from " + req.getRemoteHost() + " at " + currentTime);
+      log.info("new post from " + req.getRemoteHost() + " at " + currentTime);
       java.io.InputStream in = req.getInputStream();
 
       ServletOutputStream l_out = resp.getOutputStream();
@@ -185,6 +185,8 @@
   protected void doGet(HttpServletRequest req, HttpServletResponse resp)
       throws ServletException, IOException {
 
+
+    log.info("new GET from " + req.getRemoteHost() + " at " + System.currentTimeMillis());
     PrintStream out = new PrintStream(resp.getOutputStream());
     resp.setStatus(200);
 

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/controller/ChukwaAgentController.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/controller/ChukwaAgentController.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/controller/ChukwaAgentController.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/datacollection/controller/ChukwaAgentController.java Fri Jun 12 22:11:01 2009
@@ -89,23 +89,23 @@
   static int numArgs = 0;
 
   class Adaptor {
-    public long id = -1;
-    final public String name;
+    public String id;
+    final public String className;
     final public String params;
     final public String appType;
     public long offset;
 
-    Adaptor(String adaptorName, String appType, String params, long offset) {
-      this.name = adaptorName;
+    Adaptor(String className, String appType, String params, long offset) {
+      this.className = className;
       this.appType = appType;
       this.params = params;
       this.offset = offset;
     }
 
-    Adaptor(long id, String adaptorName, String appType, String params,
+    Adaptor(String id, String className, String appType, String params,
             long offset) {
       this.id = id;
-      this.name = adaptorName;
+      this.className = className;
       this.appType = appType;
       this.params = params;
       this.offset = offset;
@@ -115,11 +115,11 @@
      * Registers this {@link Adaptor} with the agent running at the specified
      * hostname and portno
      * 
-     * @return The id number of the this {@link Adaptor}, assigned by the agent
+     * @return The id of the this {@link Adaptor}, assigned by the agent
      *         upon successful registration
      * @throws IOException
      */
-    long register() throws IOException {
+    String register() throws IOException {
       Socket s = new Socket(hostname, portno);
       try {
         s.setSoTimeout(60000);
@@ -129,7 +129,7 @@
       }
       PrintWriter bw = new PrintWriter(new OutputStreamWriter(s
           .getOutputStream()));
-      bw.println("ADD " + name + " " + appType + " " + params + " " + offset);
+      bw.println("ADD " + className + " " + appType + " " + params + " " + offset);
       bw.flush();
       BufferedReader br = new BufferedReader(new InputStreamReader(s
           .getInputStream()));
@@ -137,10 +137,7 @@
       if (resp != null) {
         String[] fields = resp.split(" ");
         if (fields[0].equals("OK")) {
-          try {
-            id = Long.parseLong(fields[fields.length - 1]);
-          } catch (NumberFormatException e) {
-          }
+            id = fields[fields.length - 1];
         }
       }
       s.close();
@@ -179,22 +176,22 @@
     }
 
     public String toString() {
-      String[] namePieces = name.split("\\.");
+      String[] namePieces = className.split("\\.");
       String shortName = namePieces[namePieces.length - 1];
       return id + " " + shortName + " " + appType + " " + params + " " + offset;
     }
   }
 
-  Map<Long, ChukwaAgentController.Adaptor> runningAdaptors = new HashMap<Long, Adaptor>();
-  Map<Long, ChukwaAgentController.Adaptor> runningInstanceAdaptors = new HashMap<Long, Adaptor>();
-  Map<Long, ChukwaAgentController.Adaptor> pausedAdaptors;
+  Map<String, ChukwaAgentController.Adaptor> runningAdaptors = new HashMap<String, Adaptor>();
+  Map<String, ChukwaAgentController.Adaptor> runningInstanceAdaptors = new HashMap<String, Adaptor>();
+  Map<String, ChukwaAgentController.Adaptor> pausedAdaptors;
   String hostname;
   int portno;
 
   public ChukwaAgentController() {
     portno = DEFAULT_PORT;
     hostname = DEFAULT_HOST;
-    pausedAdaptors = new HashMap<Long, Adaptor>();
+    pausedAdaptors = new HashMap<String, Adaptor>();
 
     syncWithAgent();
   }
@@ -202,7 +199,7 @@
   public ChukwaAgentController(String hostname, int portno) {
     this.hostname = hostname;
     this.portno = portno;
-    pausedAdaptors = new HashMap<Long, Adaptor>();
+    pausedAdaptors = new HashMap<String, Adaptor>();
 
     syncWithAgent();
   }
@@ -219,7 +216,7 @@
       // e.printStackTrace();
       // if we can't connect to the LocalAgent, reset/clear our local view of
       // the Adaptors.
-      runningAdaptors = new HashMap<Long, ChukwaAgentController.Adaptor>();
+      runningAdaptors = new HashMap<String, ChukwaAgentController.Adaptor>();
       return false;
     }
   }
@@ -233,7 +230,7 @@
    * 
    * @return the id number of the adaptor, generated by the agent
    */
-  public long add(String adaptorName, String type, String params, long offset) {
+  public String add(String adaptorName, String type, String params, long offset) {
     return add(adaptorName, type, params, offset, 20, 15 * 1000);// retry for
                                                                  // five
                                                                  // minutes,
@@ -249,16 +246,16 @@
    * 
    * @return the id number of the adaptor, generated by the agent
    */
-  public long add(String adaptorName, String type, String params, long offset,
+  public String add(String adaptorName, String type, String params, long offset,
       long numRetries, long retryInterval) {
     ChukwaAgentController.Adaptor adaptor = new ChukwaAgentController.Adaptor(
         adaptorName, type, params, offset);
-    long adaptorID = -1;
+    String adaptorID = null;
     if (numRetries >= 0) {
       try {
         adaptorID = adaptor.register();
 
-        if (adaptorID > 0) {
+        if (adaptorID != null) {
           runningAdaptors.put(adaptorID, adaptor);
           runningInstanceAdaptors.put(adaptorID, adaptor);
         } else {
@@ -285,7 +282,7 @@
     return adaptorID;
   }
 
-  public synchronized ChukwaAgentController.Adaptor remove(long adaptorID)
+  public synchronized ChukwaAgentController.Adaptor remove(String adaptorID)
       throws IOException {
     syncWithAgent();
     ChukwaAgentController.Adaptor a = runningAdaptors.remove(adaptorID);
@@ -303,7 +300,7 @@
     // get its id, tell it to unregister itself with the agent,
     // then remove it from the list of adaptors
     for (Adaptor a : runningAdaptors.values()) {
-      if (a.name.equals(className) && a.params.equals(filename)
+      if (a.className.equals(className) && a.params.equals(filename)
           && a.appType.equals(appType)) {
         remove(a.id);
       }
@@ -312,9 +309,10 @@
 
   public void removeAll() {
     syncWithAgent();
-    Long[] keyset = runningAdaptors.keySet().toArray(new Long[] {});
+    ArrayList<String> keyset = new ArrayList<String>();
+    keyset.addAll( runningAdaptors.keySet());
 
-    for (long id : keyset) {
+    for (String id : keyset) {
       try {
         remove(id);
       } catch (IOException ioe) {
@@ -332,7 +330,7 @@
     // adaptor after agent crashes.
     for (Adaptor a : runningInstanceAdaptors.values()) {
       try {
-        remove(a.name, a.appType, a.params);
+        remove(a.className, a.appType, a.params);
       } catch (IOException ioe) {
         System.out.println("Error removing an adaptor in removeInstanceAdaptors()");
         ioe.printStackTrace();
@@ -340,7 +338,7 @@
     }
   }
 
-  Map<Long, ChukwaAgentController.Adaptor> list() throws IOException {
+  Map<String, ChukwaAgentController.Adaptor> list() throws IOException {
     Socket s = new Socket(hostname, portno);
     try {
       s.setSoTimeout(60000);
@@ -356,7 +354,7 @@
     BufferedReader br = new BufferedReader(new InputStreamReader(s
         .getInputStream()));
     String ln;
-    Map<Long, Adaptor> listResult = new HashMap<Long, Adaptor>();
+    Map<String, Adaptor> listResult = new HashMap<String, Adaptor>();
     while ((ln = br.readLine()) != null) {
       if (ln.equals("")) {
         break;
@@ -364,8 +362,7 @@
         String[] parts = ln.split("\\s+");
         if (parts.length >= 4) { // should have id, className appType, params,
                                  // offset
-          long id = Long
-              .parseLong(parts[0].substring(0, parts[0].length() - 1)); // chop
+          String id = parts[0].substring(0, parts[0].length() - 1); // chop
                                                                         // off
                                                                         // the
                                                                         // right
@@ -403,7 +400,7 @@
    * @param filename of the file for the tail adaptor to start monitoring
    * @return the id number of the adaptor, generated by the agent
    */
-  public long addFile(String appType, String filename, long numRetries,
+  public String addFile(String appType, String filename, long numRetries,
       long retryInterval) {
     filename = new File(filename).getAbsolutePath();
     // TODO: Mabye we want to check to see if the file exists here?
@@ -414,7 +411,7 @@
     // another adaptor for it
     boolean isDuplicate = false;
     for (Adaptor a : runningAdaptors.values()) {
-      if (a.name.equals(DEFAULT_FILE_TAILER) && a.appType.equals(appType)
+      if (a.className.equals(DEFAULT_FILE_TAILER) && a.appType.equals(appType)
           && a.params.endsWith(filename)) {
         isDuplicate = true;
       }
@@ -426,11 +423,11 @@
       System.out.println("An adaptor for filename \"" + filename
           + "\", type \"" + appType
           + "\", exists already, addFile() command aborted");
-      return -1;
+      return null;
     }
   }
 
-  public long addFile(String appType, String filename) {
+  public String addFile(String appType, String filename) {
     return addFile(appType, filename, 0, 0);
   }
 
@@ -446,15 +443,15 @@
    *         state of the formerly paused adaptors
    * @throws IOException
    */
-  public Collection<Long> pauseFile(String appType, String filename)
+  public Collection<String> pauseFile(String appType, String filename)
       throws IOException {
     syncWithAgent();
     // store the unique streamid of the file we are pausing.
     // search the list of adaptors for this filename
     // store the current offset for it
-    List<Long> results = new ArrayList<Long>();
+    List<String> results = new ArrayList<String>();
     for (Adaptor a : runningAdaptors.values()) {
-      if (a.name.equals(DEFAULT_FILE_TAILER) && a.params.endsWith(filename)
+      if (a.className.equals(DEFAULT_FILE_TAILER) && a.params.endsWith(filename)
           && a.appType.equals(appType)) {
         pausedAdaptors.put(a.id, a); // add it to our list of paused adaptors
         remove(a.id); // tell the agent to remove/unregister it
@@ -466,7 +463,7 @@
 
   public boolean isFilePaused(String appType, String filename) {
     for (Adaptor a : pausedAdaptors.values()) {
-      if (a.name.equals(DEFAULT_FILE_TAILER) && a.params.endsWith(filename)
+      if (a.className.equals(DEFAULT_FILE_TAILER) && a.params.endsWith(filename)
           && a.appType.equals(appType)) {
         return true;
       }
@@ -484,15 +481,15 @@
    *         old adaptors left off
    * @throws IOException
    */
-  public Collection<Long> resumeFile(String appType, String filename)
+  public Collection<String> resumeFile(String appType, String filename)
       throws IOException {
     syncWithAgent();
     // search for a record of this paused file
-    List<Long> results = new ArrayList<Long>();
+    List<String> results = new ArrayList<String>();
     for (Adaptor a : pausedAdaptors.values()) {
-      if (a.name.equals(DEFAULT_FILE_TAILER) && a.params.endsWith(filename)
+      if (a.className.equals(DEFAULT_FILE_TAILER) && a.params.endsWith(filename)
           && a.appType.equals(appType)) {
-        long newID = add(DEFAULT_FILE_TAILER, a.appType, a.offset + " "
+        String newID = add(DEFAULT_FILE_TAILER, a.appType, a.offset + " "
             + filename, a.offset);
         pausedAdaptors.remove(a.id);
         a.id = newID;
@@ -508,7 +505,7 @@
     // get its id, tell it to unregister itself with the agent,
     // then remove it from the list of adaptors
     for (Adaptor a : runningAdaptors.values()) {
-      if (a.name.equals(DEFAULT_FILE_TAILER) && a.params.endsWith(filename)
+      if (a.className.equals(DEFAULT_FILE_TAILER) && a.params.endsWith(filename)
           && a.appType.equals(appType)) {
         remove(a.id);
       }
@@ -561,11 +558,11 @@
     return new ChukwaAgentController(hostname, portno);
   }
 
-  private static long doAddFile(ChukwaAgentController c, String appType,
+  private static String doAddFile(ChukwaAgentController c, String appType,
       String params) {
     System.out.println("Adding adaptor with filename: " + params);
-    long adaptorID = c.addFile(appType, params);
-    if (adaptorID != -1) {
+    String adaptorID = c.addFile(appType, params);
+    if (adaptorID != null) {
       System.out.println("Successfully added adaptor, id is:" + adaptorID);
     } else {
       System.err

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/log4j/ChukwaDailyRollingFileAppender.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/log4j/ChukwaDailyRollingFileAppender.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/log4j/ChukwaDailyRollingFileAppender.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/log4j/ChukwaDailyRollingFileAppender.java Fri Jun 12 22:11:01 2009
@@ -542,7 +542,7 @@
             }
 
 
-            long adaptorID = chukwaClient.add(ChukwaAgentController.CharFileTailUTF8NewLineEscaped,
+            String adaptorID = chukwaClient.add(ChukwaAgentController.CharFileTailUTF8NewLineEscaped,
                 recordType,currentLength + " " + log4jFileName, currentLength,
                 numRetries, retryInterval);
 
@@ -550,7 +550,7 @@
             clientFinalizer = new ClientFinalizer(chukwaClient);
             Runtime.getRuntime().addShutdownHook(clientFinalizer);
 
-            if (adaptorID > 0) {
+            if (adaptorID != null) {
               log.debug("Added file tailing adaptor to chukwa agent for file "
                   + log4jFileName + ", adaptorId:" + adaptorID 
                   + " using this recordType :" + recordType 

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/log4j/TaskLogAppender.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/log4j/TaskLogAppender.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/log4j/TaskLogAppender.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/inputtools/log4j/TaskLogAppender.java Fri Jun 12 22:11:01 2009
@@ -57,13 +57,13 @@
             long currentLength = 0L;
             chukwaClient = new ChukwaAgentController();
             chukwaClientIsNull = false;
-            long adaptorID = chukwaClient.add(ChukwaAgentController.CharFileTailUTF8NewLineEscaped,
+            String adaptorID = chukwaClient.add(ChukwaAgentController.CharFileTailUTF8NewLineEscaped,
               recordType,currentLength + " " + log4jFileName, currentLength);
 
             // Setup a shutdownHook for the controller
             clientFinalizer = new ClientFinalizer(chukwaClient);
             Runtime.getRuntime().addShutdownHook(clientFinalizer);
-            if (adaptorID > 0) {
+            if (adaptorID != null) {
               log.debug("Added file tailing adaptor to chukwa agent for file "
                   + log4jFileName + ", adaptorId:" + adaptorID
                   + " using this recordType :" + recordType

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/tools/backfilling/BackfillingLoader.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/tools/backfilling/BackfillingLoader.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/tools/backfilling/BackfillingLoader.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/tools/backfilling/BackfillingLoader.java Fri Jun 12 22:11:01 2009
@@ -54,7 +54,7 @@
     File file = new File(logFile);
     connector.start();
     Adaptor adaptor = AdaptorFactory.createAdaptor(adaptorName);
-    adaptor.start(System.currentTimeMillis(), recordType, "0 " +file.getAbsolutePath()  ,
+    adaptor.start("", recordType, "0 " +file.getAbsolutePath(),
         0l,queue, AdaptorManager.NULL );
     adaptor.shutdown(AdaptorShutdownPolicy.WAIT_TILL_FINISHED);
     connector.shutdown();

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/util/ConstRateAdaptor.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/util/ConstRateAdaptor.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/util/ConstRateAdaptor.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/util/ConstRateAdaptor.java Fri Jun 12 22:11:01 2009
@@ -35,7 +35,7 @@
   private long offset;
   private int bytesPerSec;
   private ChunkReceiver dest;
-  private long adaptorID;
+  private String adaptorID;
 
   private volatile boolean stopping = false;
 
@@ -43,7 +43,7 @@
     return type.trim() + " " + bytesPerSec + " " + offset;
   }
 
-  public void start(long adaptor, String type, String bytesPerSecParam,
+  public void start(String adaptorID, String type, String bytesPerSecParam,
       long offset, ChunkReceiver dest, AdaptorManager c) throws AdaptorException {
     try {
       bytesPerSec = Integer.parseInt(bytesPerSecParam.trim());
@@ -51,7 +51,7 @@
       throw new AdaptorException("bad argument to const rate adaptor: ["
           + bytesPerSecParam + "]");
     }
-    this.adaptorID = adaptor;
+    this.adaptorID = adaptorID;
     this.offset = offset;
     this.type = type;
     this.dest = dest;

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/util/MaxRateSender.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/util/MaxRateSender.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/util/MaxRateSender.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/chukwa/util/MaxRateSender.java Fri Jun 12 22:11:01 2009
@@ -34,16 +34,16 @@
   private long offset;
   private String type;
   ChunkReceiver dest;
-  private long adaptorID;
+  private String adaptorID;
 
   public String getCurrentStatus() throws AdaptorException {
     return "";
   }
 
-  public void start(long adaptor, String type, String status, long offset,
+  public void start(String adaptorID, String type, String status, long offset,
       ChunkReceiver dest, AdaptorManager c) throws AdaptorException {
     this.setName("MaxRateSender adaptor");
-    this.adaptorID = adaptor;
+    this.adaptorID = adaptorID;
     this.offset = offset;
     this.type = type;
     this.dest = dest;

Modified: hadoop/chukwa/trunk/src/java/org/apache/hadoop/mapred/ChukwaJobTrackerInstrumentation.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/java/org/apache/hadoop/mapred/ChukwaJobTrackerInstrumentation.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/java/org/apache/hadoop/mapred/ChukwaJobTrackerInstrumentation.java (original)
+++ hadoop/chukwa/trunk/src/java/org/apache/hadoop/mapred/ChukwaJobTrackerInstrumentation.java Fri Jun 12 22:11:01 2009
@@ -36,7 +36,7 @@
   protected final JobTracker tracker;
   private static ChukwaAgentController chukwaClient = null;
   private static Log log = LogFactory.getLog(JobTrackerInstrumentation.class);
-  private static HashMap<JobID, Long> jobHistories = null;
+  private static HashMap<JobID, String> jobHistories = null;
 
   public ChukwaJobTrackerInstrumentation(JobTracker jt, JobConf conf) {
     super(jt, conf);
@@ -45,7 +45,7 @@
       chukwaClient = new ChukwaAgentController();
     }
     if (jobHistories == null) {
-      jobHistories = new HashMap<JobID, Long>();
+      jobHistories = new HashMap<JobID, String>();
     }
   }
 
@@ -59,7 +59,7 @@
       Path jobHistoryPath = JobHistory.JobInfo
           .getJobHistoryLogLocation(jobFileName);
       String jobConfPath = JobHistory.JobInfo.getLocalJobFilePath(id);
-      long adaptorID = chukwaClient
+      String adaptorID = chukwaClient
           .add(
               "org.apache.hadoop.chukwa.datacollection.adaptor.FileAdaptor",
               "JobConf", "0 " + jobConfPath, 0);

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/database/TestDatabaseIostat.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/database/TestDatabaseIostat.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/database/TestDatabaseIostat.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/database/TestDatabaseIostat.java Fri Jun 12 22:11:01 2009
@@ -257,10 +257,10 @@
         fullPath.append(File.separator);        
         fullPath.append(fname);
         String recordType = fname.substring(0,fname.indexOf("."));
-        long adaptorId = cli.add(
+        String adaptorId = cli.add(
           "org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8NewLineEscaped", 
           recordType, "0 " + fullPath.toString(), 0);
-        assertTrue(adaptorId != -1);
+        assertNotNull(adaptorId);
         Thread.sleep(2000);
       }
       cli.removeAll();

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/TestExecAdaptor.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/TestExecAdaptor.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/TestExecAdaptor.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/TestExecAdaptor.java Fri Jun 12 22:11:01 2009
@@ -35,9 +35,9 @@
   public void testWithPs() throws ChukwaAgent.AlreadyRunningException {
     try {
       ChukwaAgent agent = new ChukwaAgent();
-      agent
-          .processAddCommand("add org.apache.hadoop.chukwa.datacollection.adaptor.ExecAdaptor ps ps aux 0");
-
+      String psAgentID = agent.processAddCommand(
+          "add org.apache.hadoop.chukwa.datacollection.adaptor.ExecAdaptor ps ps aux 0");
+      assertNotNull(psAgentID);
       Chunk c = chunks.waitForAChunk();
       System.out.println(new String(c.getData()));
     } catch (InterruptedException e) {

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestCharFileTailingAdaptorUTF8.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestCharFileTailingAdaptorUTF8.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestCharFileTailingAdaptorUTF8.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestCharFileTailingAdaptorUTF8.java Fri Jun 12 22:11:01 2009
@@ -9,6 +9,7 @@
 import org.apache.hadoop.chukwa.Chunk;
 import org.apache.hadoop.chukwa.datacollection.agent.ChukwaAgent;
 import org.apache.hadoop.chukwa.datacollection.connector.ChunkCatcherConnector;
+import org.apache.hadoop.conf.Configuration;
 
 public class TestCharFileTailingAdaptorUTF8 extends TestCase {
   ChunkCatcherConnector chunks;
@@ -20,15 +21,17 @@
 
   public void testCrSepAdaptor() throws IOException, InterruptedException,
       ChukwaAgent.AlreadyRunningException {
-    ChukwaAgent agent = new ChukwaAgent();
+    
+    Configuration conf = new Configuration();
+    conf.set("chukwaAgent.control.port", "0");
+    ChukwaAgent agent = new ChukwaAgent(conf);
     File testFile = makeTestFile("chukwaTest", 80);
-    long adaptorId = agent
-        .processAddCommand("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8"
+    String adaptorId = agent
+        .processAddCommand("add test = org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8"
             + " lines " + testFile + " 0");
-    assertTrue(adaptorId != -1);
+    assertTrue(adaptorId.equals("test"));
     System.out.println("getting a chunk...");
     Chunk c = chunks.waitForAChunk();
-    System.out.println("got chunk");
     assertTrue(c.getSeqID() == testFile.length());
 
     assertTrue(c.getRecordOffsets().length == 80);
@@ -36,7 +39,6 @@
     for (int rec = 0; rec < c.getRecordOffsets().length; ++rec) {
       String record = new String(c.getData(), recStart,
           c.getRecordOffsets()[rec] - recStart + 1);
-      System.out.println("record " + rec + " was: " + record);
       assertTrue(record.equals(rec + " abcdefghijklmnopqrstuvwxyz\n"));
       recStart = c.getRecordOffsets()[rec] + 1;
     }

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileExpirationPolicy.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileExpirationPolicy.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileExpirationPolicy.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileExpirationPolicy.java Fri Jun 12 22:11:01 2009
@@ -45,11 +45,11 @@
 
       FileTailingAdaptor.GRACEFUL_PERIOD = 30 * 1000;
 
-      long adaptorId = agent
+      String adaptorId = agent
           .processAddCommand("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8NewLineEscaped MyType 0 /myWrongPath"
               + System.currentTimeMillis() + " 0");
 
-      assertTrue(adaptorId != -1);
+      assertTrue(adaptorId != null);
 
       assertNotNull(agent.getAdaptor(adaptorId));
 
@@ -91,11 +91,11 @@
       assertTrue(testFile.canRead() == true);
 
       FileTailingAdaptor.GRACEFUL_PERIOD = 30 * 1000;
-      long adaptorId = agent
+      String adaptorId = agent
           .processAddCommand("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8NewLineEscaped MyType 0 "
               + logFile + " 0");
 
-      assertTrue(adaptorId != -1);
+      assertTrue(adaptorId != null);
 
       assertNotNull(agent.getAdaptor(adaptorId));
 

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileTailingAdaptorBigRecord.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileTailingAdaptorBigRecord.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileTailingAdaptorBigRecord.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileTailingAdaptorBigRecord.java Fri Jun 12 22:11:01 2009
@@ -63,27 +63,27 @@
       // sleep for some time to make sure we don't get chunk from existing
       // streams
       Thread.sleep(5000);
-      long adaptorId = agent
+      String adaptorId = agent
           .processAddCommand("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8NewLineEscaped"
               + " BigRecord " + logFile + " 0");
-      assertTrue(adaptorId != -1);
+      assertNotNull(adaptorId);
 
       boolean record8Found = false;
       Chunk c = null;
       // Keep reading until record8
       // If the adaptor is stopped then Junit will fail with a timeOut
       while (!record8Found) {
-        c = chunks.waitForAChunk();
+        c = chunks.waitForAChunk();//only wait three minutes
         String data = new String(c.getData());
         if (c.getDataType().equals("BigRecord")
             && data.indexOf("8 abcdefghijklmnopqrstuvwxyz") >= 0) {
           record8Found = true;
         }
       }
-      agent.getAdaptor(adaptorId).shutdown();
+      agent.stopAdaptor(adaptorId, true);
       agent.shutdown();
     } catch (Exception e) {
-      Assert.fail("Exception in testBigRecord" + e.getMessage());
+      Assert.fail("Exception in testBigRecord: " + e.getMessage());
     } finally {
       if (f != null) {
         f.delete();

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileTailingAdaptors.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileTailingAdaptors.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileTailingAdaptors.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestFileTailingAdaptors.java Fri Jun 12 22:11:01 2009
@@ -48,10 +48,10 @@
     // sleep for some time to make sure we don't get chunk from existing streams
     Thread.sleep(5000);
     File testFile = makeTestFile("chukwaCrSepTest", 80);
-    long adaptorId = agent
+    String adaptorId = agent
         .processAddCommand("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8"
             + " lines " + testFile + " 0");
-    assertTrue(adaptorId != -1);
+    assertNotNull(adaptorId);
     System.out.println("getting a chunk...");
     Chunk c = chunks.waitForAChunk();
     System.out.println("got chunk");

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestLogRotate.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestLogRotate.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestLogRotate.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestLogRotate.java Fri Jun 12 22:11:01 2009
@@ -48,10 +48,10 @@
     // sleep for some time to make sure we don't get chunk from existing streams
     Thread.sleep(5000);
     File testFile = makeTestFile("chukwaLogRotateTest", 80);
-    long adaptorId = agent
-        .processAddCommand("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8"
+    String adaptorId = agent
+        .processAddCommand("add lr =org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8"
             + " lines " + testFile + " 0");
-    assertTrue(adaptorId != -1);
+    assertTrue(adaptorId.equals("lr"));
     System.out.println("getting a chunk...");
     Chunk c = chunks.waitForAChunk();
     System.out.println("got chunk");

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestRawAdaptor.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestRawAdaptor.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestRawAdaptor.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestRawAdaptor.java Fri Jun 12 22:11:01 2009
@@ -49,10 +49,10 @@
     // sleep for some time to make sure we don't get chunk from existing streams
     Thread.sleep(5000);
     File testFile = makeTestFile("chukwaRawTest", 80);
-    long adaptorId = agent
+    String adaptorId = agent
         .processAddCommand("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.FileTailingAdaptor"
             + " raw " + testFile + " 0");
-    assertTrue(adaptorId != -1);
+    assertNotNull(adaptorId);
     Chunk c = chunks.waitForAChunk();
     while (!c.getDataType().equals("raw")) {
       c = chunks.waitForAChunk();

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestStartAtOffset.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestStartAtOffset.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestStartAtOffset.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/adaptor/filetailer/TestStartAtOffset.java Fri Jun 12 22:11:01 2009
@@ -51,10 +51,10 @@
     Thread.sleep(5000);
     File testFile = makeTestFile();
     int startOffset = 0; // skip first line
-    long adaptorId = agent
+    String adaptorId = agent
         .processAddCommand("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8 "
             + "lines " + startOffset + " " + testFile + " " + startOffset);
-    assertTrue(adaptorId != -1);
+    assertTrue(adaptorId != null);
     System.out.println("getting a chunk...");
     Chunk c = chunks.waitForAChunk();
     System.out.println("got chunk");
@@ -89,7 +89,7 @@
     Thread.sleep(5000);
     File testFile = makeTestFile();
     int startOffset = 0;
-    long adaptorId = agent
+    String adaptorId = agent
         .processAddCommand("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8 "
             + "lines "
             + startOffset
@@ -97,7 +97,7 @@
             + testFile
             + " "
             + (startOffset + 29));
-    assertTrue(adaptorId != -1);
+    assertTrue(adaptorId != null);
     System.out.println("getting a chunk...");
     Chunk c = chunks.waitForAChunk();
     System.out.println("got chunk");

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestAgent.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestAgent.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestAgent.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestAgent.java Fri Jun 12 22:11:01 2009
@@ -39,10 +39,10 @@
       ChukwaAgentController cli = new ChukwaAgentController("localhost", portno);
 
       for (int i = 1; i < 20; ++i) {
-        long adaptorId = cli.add(
+        String adaptorId = cli.add(
             "org.apache.hadoop.chukwa.util.ConstRateAdaptor", "raw" + i, "2000"
                 + i, 0);
-        assertTrue(adaptorId != -1);
+        assertNotNull(adaptorId);
         Thread.sleep(2000);
         cli.removeAll();
       }
@@ -62,17 +62,17 @@
       conn.start();
       int count = agent.adaptorCount();
       for (int trial = 0; trial < 20; ++trial) {
-        ArrayList<Long> runningAdaptors = new ArrayList<Long>();
+        ArrayList<String> runningAdaptors = new ArrayList<String>();
 
         for (int i = 1; i < 7; ++i) {
-          long l = agent
+          String l = agent
               .processAddCommand("add  org.apache.hadoop.chukwa.util.ConstRateAdaptor  raw"
                   + i + " 2000" + i + " 0");
-          assertTrue(l != -1);
+          assertTrue(l != null);
           runningAdaptors.add(l);
         }
         Thread.sleep(1000);
-        for (Long l : runningAdaptors)
+        for (String l : runningAdaptors)
           agent.stopAdaptor(l, false);
         Thread.sleep(5000);
         assertTrue(agent.adaptorCount() == count);

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestAgentConfig.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestAgentConfig.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestAgentConfig.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestAgentConfig.java Fri Jun 12 22:11:01 2009
@@ -26,8 +26,7 @@
       File initialAdaptors = File.createTempFile("initial", "adaptors");
       initialAdaptors.deleteOnExit();
       ps = new PrintStream(new FileOutputStream(initialAdaptors));
-      ps
-          .println("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8  raw 0 "
+      ps.println("add testAdaptor= org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8  raw 0 "
               + foo.getAbsolutePath() + " 0  ");
       ps.close();
 
@@ -45,8 +44,8 @@
       conn.start();
       assertEquals(1, agent.adaptorCount());// check that we processed initial
                                             // adaptors
-      assertNotNull(agent.getAdaptor(1L));
-      assertTrue(agent.getAdaptor(1L).getStreamName().contains("foo"));
+      assertNotNull(agent.getAdaptor("testAdaptor"));
+      assertTrue(agent.getAdaptor("testAdaptor").getStreamName().contains("foo"));
 
       System.out
           .println("---------------------done with first run, now stopping");
@@ -58,8 +57,7 @@
       // we should
       // still only be looking at foo.
       ps = new PrintStream(new FileOutputStream(initialAdaptors, false));// overwrite
-      ps
-          .println("add org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8  raw 0 "
+      ps.println("add bar= org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8  raw 0 "
               + bar.getAbsolutePath() + " 0  ");
       ps.close();
 
@@ -69,8 +67,8 @@
       conn.start();
       assertEquals(1, agent.adaptorCount());// check that we processed initial
                                             // adaptors
-      assertNotNull(agent.getAdaptor(1L));
-      assertTrue(agent.getAdaptor(1L).getStreamName().contains("foo"));
+      assertNotNull(agent.getAdaptor("testAdaptor"));
+      assertTrue(agent.getAdaptor("testAdaptor").getStreamName().contains("foo"));
       agent.shutdown();
       System.out.println("---------------------done");
 
@@ -116,8 +114,7 @@
       (new File(NONCE_DIR, "chukwa_checkpoint_0")).createNewFile();
       agent = new ChukwaAgent(conf);
       assertEquals(0, agent.getAdaptorList().size());
-      agent
-          .processAddCommand("ADD org.apache.hadoop.chukwa.datacollection.adaptor.ChukwaTestAdaptor testdata  0");
+      agent.processAddCommand("ADD org.apache.hadoop.chukwa.datacollection.adaptor.ChukwaTestAdaptor testdata  0");
       agent.shutdown();
       assertTrue(new File(NONCE_DIR, "chukwa_checkpoint_1").exists());
 

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestCmd.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestCmd.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestCmd.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/agent/TestCmd.java Fri Jun 12 22:11:01 2009
@@ -33,9 +33,9 @@
       agent = new ChukwaAgent();
       ConsoleOutConnector conn = new ConsoleOutConnector(agent, true);
       conn.start();
-      long l = agent
+      String l = agent
           .processAddCommand("ADD org.apache.hadoop.chukwa.datacollection.adaptor.ChukwaTestAdaptor  chukwaTestAdaptorType 0 my param1 param2 /var/log/messages 114027");
-      assertTrue(l != -1);
+      assertTrue(l != null);
       Adaptor adaptor = agent.getAdaptor(l);
       ChukwaTestAdaptor chukwaTestAdaptor = (ChukwaTestAdaptor) adaptor;
       assertTrue("error in type",
@@ -63,10 +63,10 @@
       agent = new ChukwaAgent();
       ConsoleOutConnector conn = new ConsoleOutConnector(agent, true);
       conn.start();
-      long l = agent
+      String name = agent
           .processAddCommand("ADD org.apache.hadoop.chukwa.datacollection.adaptor.ChukwaTestAdaptor  chukwaTestAdaptorType 114027");
-      assertTrue(l != -1);
-      Adaptor adaptor = agent.getAdaptor(l);
+      assertTrue(name != null);
+      Adaptor adaptor = agent.getAdaptor(name);
       ChukwaTestAdaptor chukwaTestAdaptor = (ChukwaTestAdaptor) adaptor;
       assertTrue("error in type",
           "chukwaTestAdaptorType".intern() == chukwaTestAdaptor.getType()
@@ -75,7 +75,7 @@
           .intern());
       assertTrue("error in startOffset", 114027l == chukwaTestAdaptor
           .getStartOffset());
-      agent.stopAdaptor(l, false);
+      agent.stopAdaptor(name, false);
       agent.shutdown();
       Thread.sleep(2000);
     } catch (InterruptedException e) {
@@ -92,11 +92,11 @@
       agent = new ChukwaAgent();
       ConsoleOutConnector conn = new ConsoleOutConnector(agent, true);
       conn.start();
-      long l = agent
+      String n = agent
           .processAddCommand("ADD org.apache.hadoop.chukwa.datacollection.adaptor.ChukwaTestAdaptor"
               + "  chukwaTestAdaptorType 0  114027");
-      assertTrue(l != -1);
-      Adaptor adaptor = agent.getAdaptor(l);
+      assertTrue(n != null);
+      Adaptor adaptor = agent.getAdaptor(n);
       ChukwaTestAdaptor chukwaTestAdaptor = (ChukwaTestAdaptor) adaptor;
       assertTrue("error in type",
           "chukwaTestAdaptorType".intern() == chukwaTestAdaptor.getType()
@@ -105,7 +105,7 @@
           .getParams().intern());
       assertTrue("error in startOffset", 114027l == chukwaTestAdaptor
           .getStartOffset());
-      agent.stopAdaptor(l, false);
+      agent.stopAdaptor(n, false);
       agent.shutdown();
       Thread.sleep(2000);
     } catch (InterruptedException e) {

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/connector/ChunkCatcherConnector.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/connector/ChunkCatcherConnector.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/connector/ChunkCatcherConnector.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/connector/ChunkCatcherConnector.java Fri Jun 12 22:11:01 2009
@@ -20,23 +20,45 @@
 
 import org.apache.hadoop.chukwa.Chunk;
 import org.apache.hadoop.chukwa.datacollection.*;
-import java.util.ArrayList;
+import java.util.*;
 
 public class ChunkCatcherConnector implements Connector {
 
   ChunkQueue eq;
+  
+  Timer tm;
+  
+  class Interruptor extends TimerTask {
+    Thread targ;
+    Interruptor(Thread t) {
+      targ =t;
+    }
+    
+    public void run() {
+      targ.interrupt();
+    }
+  };
 
   public void start() {
     eq = DataFactory.getInstance().getEventQueue();
+    tm = new Timer();
   }
 
-  public Chunk waitForAChunk() throws InterruptedException {
+  public Chunk waitForAChunk(long ms) throws InterruptedException {
+    
     ArrayList<Chunk> chunks = new ArrayList<Chunk>();
+    if(ms > 0)
+      tm.schedule(new Interruptor(Thread.currentThread()), ms);
     eq.collect(chunks, 1);
     return chunks.get(0);
   }
+  
+  public Chunk waitForAChunk() throws InterruptedException {
+    return this.waitForAChunk(0);//wait forever by default
+  }
 
   public void shutdown() {
+    tm.cancel();
   }
 
   @Override

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/controller/TestAgentClient.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/controller/TestAgentClient.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/controller/TestAgentClient.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/datacollection/controller/TestAgentClient.java Fri Jun 12 22:11:01 2009
@@ -60,12 +60,12 @@
     try {
       // add the fileTailer to the agent using the client
       System.out.println("Adding adaptor with filename: " + params);
-      long adaptorID = c.addFile(appType, params);
+      String adaptorID = c.addFile(appType, params);
       System.out.println("Successfully added adaptor, id is:" + adaptorID);
 
       // do a list on the agent to see if the adaptor has been added for this
       // file
-      Map<Long, ChukwaAgentController.Adaptor> listResult = c.list();
+      Map<String, ChukwaAgentController.Adaptor> listResult = c.list();
       assertTrue(listResult.containsKey(adaptorID));
     } catch (IOException e) {
       e.printStackTrace();

Modified: hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/validationframework/ChukwaAgentToCollectorValidator.java
URL: http://svn.apache.org/viewvc/hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/validationframework/ChukwaAgentToCollectorValidator.java?rev=784294&r1=784293&r2=784294&view=diff
==============================================================================
--- hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/validationframework/ChukwaAgentToCollectorValidator.java (original)
+++ hadoop/chukwa/trunk/src/test/org/apache/hadoop/chukwa/validationframework/ChukwaAgentToCollectorValidator.java Fri Jun 12 22:11:01 2009
@@ -160,8 +160,7 @@
     // 0 /var/log/messages
     // 0
     System.out.println("Adding adaptor");
-    long adaptor = cli
-        .add(
+    String adaptor = cli.add(
             "org.apache.hadoop.chukwa.datacollection.adaptor.filetailer.CharFileTailingAdaptorUTF8NewLineEscaped",
             "AutomatedTestType", "0 " + inputFile, 0);
 



Mime
View raw message