Browse Source

HADOOP-5033 Simplify Chukwa writer API.
Contributed by Ari Rabkin (asrabkin)



git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@747979 13f79535-47bb-0310-9956-ffa450edef68

Ariel Shemaiah Rabkin 16 years ago
parent
commit
78719a6274

+ 0 - 1
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/ChukwaWriter.java

@@ -26,7 +26,6 @@ import org.apache.hadoop.conf.Configuration;
 public interface ChukwaWriter
 {
 	public void init(Configuration c) throws WriterException;
-	public void add(Chunk data) throws WriterException;
 	public void add(List<Chunk> chunks) throws WriterException;
 	public void close() throws WriterException;;
 

+ 7 - 8
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/Dedup.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.chukwa.datacollection.writer;
 
 import java.util.ArrayDeque;
 import java.util.HashSet;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Queue;
 
@@ -100,17 +101,15 @@ public class Dedup implements PipelineableWriter {
     this.next = next;
   }
 
-  @Override
-  public void add(Chunk data) throws WriterException {
-    if(! cache.addAndCheck(new DedupKey(data.getStreamName(), data.getSeqID())))
-        next.add(data);
-  }
-
   @Override
   public void add(List<Chunk> chunks) throws WriterException {
+    ArrayList<Chunk> passedThrough = new ArrayList<Chunk>();
     for(Chunk c: chunks)
-      add(c);
-
+      if(! cache.addAndCheck(new DedupKey(c.getStreamName(), c.getSeqID())))
+        passedThrough.add(c);
+    
+    if(!passedThrough.isEmpty())
+      next.add(passedThrough);
   }
 
   @Override

+ 0 - 6
src/contrib/chukwa/src/java/org/apache/hadoop/chukwa/datacollection/writer/PipelineStageWriter.java

@@ -30,12 +30,6 @@ public class PipelineStageWriter implements ChukwaWriter {
 
   ChukwaWriter writer; //head of pipeline
 
-  @Override
-  public void add(Chunk data) throws WriterException {
-    writer.add(data);
-
-  }
-
   @Override
   public void add(List<Chunk> chunks) throws WriterException {
     writer.add(chunks);

+ 4 - 9
src/contrib/chukwa/src/test/org/apache/hadoop/chukwa/datacollection/collector/CaptureWriter.java

@@ -28,18 +28,13 @@ import org.apache.hadoop.conf.Configuration;
 public class CaptureWriter implements ChukwaWriter {
   static ArrayList<Chunk> outputs = new ArrayList<Chunk>();
 
-  @Override
-  public void add(Chunk data) throws WriterException {
-    synchronized(outputs) {
-      outputs.add(data);
-    }
-    
-  }
-
   @Override
   public void add(List<Chunk> chunks) throws WriterException {
+
+    synchronized(outputs) {
      for(Chunk c: chunks)
-       add(c);
+       outputs.add(c);
+    }
   }
 
   @Override