RemoveBridge pass test and add RemoveLowCoverage and Scaffolding
diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/HashMapWritable.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/HashMapWritable.java
new file mode 100644
index 0000000..c4556c7
--- /dev/null
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/HashMapWritable.java
@@ -0,0 +1,102 @@
+package edu.uci.ics.genomix.pregelix.io;

+

+import java.io.DataInput;

+import java.io.DataOutput;

+import java.io.IOException;

+import java.util.Map;

+import java.util.Set;

+import java.util.HashMap;

+

+import org.apache.hadoop.io.Writable;

+@SuppressWarnings("unchecked") 

+public class HashMapWritable<K extends Writable, V extends Writable> extends HashMap<K, V> implements

+        Writable {

+

+	/**

+     * 

+     */

+    private static final long serialVersionUID = 1L;

+

+    /**

+	 * Creates a HashMapWritable object.

+	 */

+	public HashMapWritable() {

+		super();

+	}

+	

+	/**

+	 * Creates a HashMapWritable object from a regular HashMap.

+	 */

+	public HashMapWritable(HashMap<K, V> map) {

+		super(map);

+	}

+

+	/**

+	 * Deserializes the array.

+	 * 

+	 * @param in

+	 *            source for raw byte representation

+	 */

+

+	@SuppressWarnings("rawtypes")

+    public void readFields(DataInput in) throws IOException {

+		

+		this.clear();

+

+		int numEntries = in.readInt();

+		if(numEntries==0) return;

+		

+		String keyClassName = in.readUTF();

+		String valueClassName = in.readUTF();

+		

+		K objK;

+		V objV;

+		try {

+			Class keyClass = Class.forName(keyClassName);

+			Class valueClass = Class.forName(valueClassName);

+			for (int i = 0; i < numEntries; i++) {

+				objK = (K) keyClass.newInstance();

+				objK.readFields(in);

+				objV = (V) valueClass.newInstance();

+				objV.readFields(in);

+				put(objK, objV);

+			}

+

+		} catch (ClassNotFoundException e) {

+			e.printStackTrace();

+		} catch (IllegalAccessException e) {

+			e.printStackTrace();

+		} catch (InstantiationException e) {

+			e.printStackTrace();

+		}

+		

+	}

+

+	/**

+	 * Serializes this array.

+	 * 

+	 * @param out

+	 *            where to write the raw byte representation

+	 */

+	public void write(DataOutput out) throws IOException {

+		// Write out the number of entries in the map

+	    out.writeInt(size());

+	    if(size()==0) return;

+	    

+	    // Write out the class names for keys and values

+	    // assuming that data is homogeneuos (i.e., all entries have same types)

+	    Set<Map.Entry<K, V>> entries = entrySet();

+	    Map.Entry<K, V> first = entries.iterator().next();

+	    K objK = first.getKey();

+	    V objV = first.getValue();

+	    out.writeUTF(objK.getClass().getCanonicalName());

+	    out.writeUTF(objV.getClass().getCanonicalName());

+

+	    // Then write out each key/value pair

+	    for (Map.Entry<K, V> e: entrySet()) {

+	      e.getKey().write(out);

+	      e.getValue().write(out);

+	    }

+	}

+

+}

diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/MessageWritable.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/MessageWritable.java
index bfb1e64..822d136 100644
--- a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/MessageWritable.java
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/MessageWritable.java
@@ -172,6 +172,18 @@
         }
     }
     
+    /** for Scaffolding, startVertexId is used as seekedVertexId **/  
+    public VKmerBytesWritable getSeekedVertexId() {
+        return startVertexId;
+    }
+
+    public void setSeekedVertexId(VKmerBytesWritable startVertexId) {
+        if(startVertexId != null){
+            checkMessage |= CheckMessage.START;
+            this.startVertexId.setAsCopy(startVertexId);
+        }
+    }
+    
     public float getAverageCoverage() {
         return averageCoverage;
     }
diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/VertexValueWritable.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/VertexValueWritable.java
index f67ff6b..237d9ce 100644
--- a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/VertexValueWritable.java
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/io/VertexValueWritable.java
@@ -1,7 +1,6 @@
 package edu.uci.ics.genomix.pregelix.io;
 
 import java.io.*;
-
 import org.apache.hadoop.io.WritableComparable;
 
 import edu.uci.ics.genomix.type.PositionListWritable;
@@ -54,6 +53,7 @@
     private float averageCoverage;
     private byte state;
     private boolean isFakeVertex = false;
+    private HashMapWritable<VKmerBytesWritable, VKmerListWritable> traverseMap = new HashMapWritable<VKmerBytesWritable, VKmerListWritable>();
 
     public VertexValueWritable() {
         this(0);
@@ -97,6 +97,11 @@
     public PositionListWritable getNodeIdList() {
         return nodeIdList;
     }
+    
+    //for testing 
+    public long getHeadReadId(){
+        return 1;
+    }
 
     public void setNodeIdList(PositionListWritable nodeIdList) {
         this.nodeIdList.set(nodeIdList);
@@ -186,6 +191,15 @@
         return actualKmer.getKmerLetterLength();
     }
     
+    
+    public HashMapWritable<VKmerBytesWritable, VKmerListWritable> getTraverseMap() {
+        return traverseMap;
+    }
+
+    public void setTraverseMap(HashMapWritable<VKmerBytesWritable, VKmerListWritable> traverseMap) {
+        this.traverseMap = traverseMap;
+    }
+
     public void reset() {
         this.reset(0);
     }
@@ -212,6 +226,7 @@
         averageCoverage = in.readFloat();
         this.state = in.readByte();
         this.isFakeVertex = in.readBoolean();
+        this.traverseMap.readFields(in);
     }
 
     @Override
@@ -225,6 +240,7 @@
         out.writeFloat(averageCoverage);
         out.writeByte(this.state);
         out.writeBoolean(this.isFakeVertex);
+        this.traverseMap.write(out);
     }
 
     @Override
diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/removelowcoverage/RemoveLowCoverageVertex.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/removelowcoverage/RemoveLowCoverageVertex.java
new file mode 100644
index 0000000..3e822e4
--- /dev/null
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/removelowcoverage/RemoveLowCoverageVertex.java
@@ -0,0 +1,55 @@
+package edu.uci.ics.genomix.pregelix.operator.removelowcoverage;
+
+import java.util.Iterator;
+
+import org.apache.hadoop.io.NullWritable;
+
+import edu.uci.ics.genomix.pregelix.client.Client;
+import edu.uci.ics.genomix.pregelix.format.GraphCleanInputFormat;
+import edu.uci.ics.genomix.pregelix.format.GraphCleanOutputFormat;
+import edu.uci.ics.genomix.pregelix.io.MessageWritable;
+import edu.uci.ics.genomix.pregelix.io.VertexValueWritable;
+import edu.uci.ics.genomix.type.VKmerBytesWritable;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+
+public class RemoveLowCoverageVertex extends
+    Vertex<VKmerBytesWritable, VertexValueWritable, NullWritable, MessageWritable> {
+    public static final String KMER_SIZE = "RemoveLowCoverageVertex.kmerSize";
+    public static final String MIN_AVERAGECOVERAGE = "RemoveLowCoverageVertex.minAverageCoverage";
+    public static int kmerSize = -1;
+    private static float minAverageCoverage = -1;
+    
+    /**
+     * initiate kmerSize, length
+     */
+    public void initVertex() {
+        if (kmerSize == -1)
+            kmerSize = getContext().getConfiguration().getInt(KMER_SIZE, 5);
+        if(minAverageCoverage == -1)
+            minAverageCoverage = getContext().getConfiguration().getFloat(MIN_AVERAGECOVERAGE, 5);
+    }
+    
+    @Override
+    public void compute(Iterator<MessageWritable> msgIterator) {
+        initVertex(); 
+        if(getVertexValue().getAverageCoverage() < minAverageCoverage)
+            deleteVertex(getVertexId());
+        else
+            voteToHalt();
+    }
+    
+    public static void main(String[] args) throws Exception {
+        PregelixJob job = new PregelixJob(RemoveLowCoverageVertex.class.getSimpleName());
+        job.setVertexClass(RemoveLowCoverageVertex.class);
+        /**
+         * BinaryInput and BinaryOutput
+         */
+        job.setVertexInputFormatClass(GraphCleanInputFormat.class);
+        job.setVertexOutputFormatClass(GraphCleanOutputFormat.class);
+        job.setDynamicVertexValueSize(true);
+        job.setOutputKeyClass(VKmerBytesWritable.class);
+        job.setOutputValueClass(VertexValueWritable.class);
+        Client.run(args, job);
+    }
+}
diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/scaffolding/ScaffoldingVertex.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/scaffolding/ScaffoldingVertex.java
new file mode 100644
index 0000000..be43658
--- /dev/null
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/scaffolding/ScaffoldingVertex.java
@@ -0,0 +1,89 @@
+package edu.uci.ics.genomix.pregelix.operator.scaffolding;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import edu.uci.ics.genomix.pregelix.io.HashMapWritable;
+import edu.uci.ics.genomix.pregelix.io.MessageWritable;
+import edu.uci.ics.genomix.pregelix.operator.pathmerge.MapReduceVertex;
+import edu.uci.ics.genomix.type.VKmerBytesWritable;
+import edu.uci.ics.genomix.type.VKmerListWritable;
+
+public class ScaffoldingVertex extends 
+    MapReduceVertex{
+
+    public static Map<Long, VKmerListWritable> scaffoldingMap = new HashMap<Long, VKmerListWritable>();
+    
+    private HashMapWritable<VKmerBytesWritable, VKmerListWritable> traverseMap = new HashMapWritable<VKmerBytesWritable, VKmerListWritable>();
+    
+    public void initVertex() {
+        if (kmerSize == -1)
+            kmerSize = getContext().getConfiguration().getInt(KMER_SIZE, 5);
+        if (maxIteration < 0)
+            maxIteration = getContext().getConfiguration().getInt(ITERATIONS, 1000000);
+        if(incomingMsg == null)
+            incomingMsg = new MessageWritable(kmerSize);
+        if(outgoingMsg == null)
+            outgoingMsg = new MessageWritable(kmerSize);
+        else
+            outgoingMsg.reset(kmerSize);
+        if(fakeVertex == null){
+            fakeVertex = new VKmerBytesWritable();
+            String random = generaterRandomString(kmerSize + 1);
+            fakeVertex.setByRead(kmerSize + 1, random.getBytes(), 0); 
+        }
+        if(destVertexId == null)
+            destVertexId = new VKmerBytesWritable(kmerSize);
+        if(kmerList == null)
+            kmerList = new VKmerListWritable();
+    }
+    
+    @Override
+    public void compute(Iterator<MessageWritable> msgIterator) {
+        initVertex();
+        if(getSuperstep() == 1){
+            /** add a fake vertex **/
+            addFakeVertex();
+            /** grouped by 5' readId **/
+            long mainReadId = getVertexValue().getHeadReadId();
+            if(mainReadId != 0){ //empty or not
+                if(scaffoldingMap.containsKey(mainReadId)){
+                    kmerList.setCopy(scaffoldingMap.get(mainReadId));
+                    kmerList.append(getVertexId());
+                } else{
+                    kmerList.reset();
+                    kmerList.append(getVertexId());
+                }
+                scaffoldingMap.put(mainReadId, kmerList);
+            }
+            voteToHalt();
+        } else if(getSuperstep() == 2){
+            /** process scaffoldingMap **/
+            for(Long readId : scaffoldingMap.keySet()){
+                kmerList.setCopy(scaffoldingMap.get(readId));
+                if(kmerList.getCountOfPosition() == 2){
+                    outgoingMsg.setSeekedVertexId(kmerList.getPosition(1));
+                    sendMsg(kmerList.getPosition(0), outgoingMsg);
+                    outgoingMsg.setSeekedVertexId(kmerList.getPosition(0));
+                    sendMsg(kmerList.getPosition(1), outgoingMsg);
+                }
+            }
+            deleteVertex(getVertexId());
+        } else if(getSuperstep() == 3){
+            if(msgIterator.hasNext()){
+                incomingMsg = msgIterator.next();
+                
+                /** initiate the traverseMap in vertexValue **/
+                kmerList.reset();
+                kmerList.append(incomingMsg.getSeekedVertexId());
+                traverseMap.clear();
+                traverseMap.put(incomingMsg.getSeekedVertexId(), kmerList);
+                getVertexValue().setTraverseMap(traverseMap);
+                
+                /** begin to traverse **/
+                
+            }
+        }
+    }
+}
diff --git a/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobGen/JobGenerator.java b/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobGen/JobGenerator.java
index 4f0f39b..98219d7 100644
--- a/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobGen/JobGenerator.java
+++ b/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobGen/JobGenerator.java
@@ -16,6 +16,8 @@
 import edu.uci.ics.genomix.pregelix.operator.pathmerge.P2ForPathMergeVertex;
 import edu.uci.ics.genomix.pregelix.operator.pathmerge.MapReduceVertex;
 import edu.uci.ics.genomix.pregelix.operator.pathmerge.P4ForPathMergeVertex;
+import edu.uci.ics.genomix.pregelix.operator.removelowcoverage.RemoveLowCoverageVertex;
+import edu.uci.ics.genomix.pregelix.operator.scaffolding.ScaffoldingVertex;
 import edu.uci.ics.genomix.pregelix.operator.splitrepeat.SplitRepeatVertex;
 import edu.uci.ics.genomix.pregelix.operator.tipremove.TipAddVertex;
 import edu.uci.ics.genomix.pregelix.operator.tipremove.TipRemoveVertex;
@@ -25,7 +27,23 @@
 public class JobGenerator {
 
     public static String outputBase = "src/test/resources/jobs/";
+    
+    private static void generateMapReduceGraphJob(String jobName, String outputPath) throws IOException {
+        PregelixJob job = new PregelixJob(jobName);
+        job.setVertexClass(MapReduceVertex.class);
+        job.setVertexInputFormatClass(GraphCleanInputFormat.class);
+        job.setVertexOutputFormatClass(P2PathMergeOutputFormat.class); 
+        job.setDynamicVertexValueSize(true);
+        job.setOutputKeyClass(VKmerBytesWritable.class);
+        job.setOutputValueClass(VertexValueWritable.class);
+        job.getConfiguration().setInt(MapReduceVertex.KMER_SIZE, 3);
+        job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+    }
 
+    private static void genMapReduceGraph() throws IOException {
+        generateMapReduceGraphJob("MapReduceGraph", outputBase + "MapReduceGraph.xml");
+    }
+    
 //    private static void generateNaiveAlgorithmForMergeGraphJob(String jobName, String outputPath) throws IOException {
 //        PregelixJob job = new PregelixJob(jobName);
 //        job.setVertexClass(P1ForPathMergeVertex.class);
@@ -95,37 +113,23 @@
                 + "P4ForMergeGraph.xml");
     }
     
-    private static void generateMapReduceGraphJob(String jobName, String outputPath) throws IOException {
+    private static void generateRemoveLowCoverageGraphJob(String jobName, String outputPath) throws IOException {
         PregelixJob job = new PregelixJob(jobName);
-        job.setVertexClass(MapReduceVertex.class);
+        job.setVertexClass(RemoveLowCoverageVertex.class);
         job.setVertexInputFormatClass(GraphCleanInputFormat.class);
-        job.setVertexOutputFormatClass(P2PathMergeOutputFormat.class); 
+        job.setVertexOutputFormatClass(GraphCleanOutputFormat.class);
         job.setDynamicVertexValueSize(true);
         job.setOutputKeyClass(VKmerBytesWritable.class);
         job.setOutputValueClass(VertexValueWritable.class);
-        job.getConfiguration().setInt(MapReduceVertex.KMER_SIZE, 3);
+        job.getConfiguration().setInt(RemoveLowCoverageVertex.KMER_SIZE, 3);
         job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
     }
 
-    private static void genMapReduceGraph() throws IOException {
-        generateMapReduceGraphJob("MapReduceGraph", outputBase + "MapReduceGraph.xml");
+    private static void genRemoveLowCoverageGraph() throws IOException {
+        generateRemoveLowCoverageGraphJob("RemoveLowCoverageGraph", outputBase
+                + "RemoveLowCoverageGraph.xml");
     }
     
-    private static void generateSplitRepeatGraphJob(String jobName, String outputPath) throws IOException {
-        PregelixJob job = new PregelixJob(jobName);
-        job.setVertexClass(SplitRepeatVertex.class);
-        job.setVertexInputFormatClass(InitialGraphCleanInputFormat.class);
-        job.setVertexOutputFormatClass(GraphCleanOutputFormat.class); 
-        job.setDynamicVertexValueSize(true);
-        job.setOutputKeyClass(VKmerBytesWritable.class);
-        job.setOutputValueClass(VertexValueWritable.class);
-        job.getConfiguration().setInt(SplitRepeatVertex.KMER_SIZE, 3);
-        job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
-    }
-
-    private static void genSplitRepeatGraph() throws IOException {
-        generateSplitRepeatGraphJob("SplitRepeatGraph", outputBase + "SplitRepeatGraph.xml");
-    }
     private static void generateTipAddGraphJob(String jobName, String outputPath) throws IOException {
         PregelixJob job = new PregelixJob(jobName);
         job.setVertexClass(TipAddVertex.class);
@@ -160,6 +164,23 @@
                 + "TipRemoveGraph.xml");
     }
     
+    private static void generateSplitRepeatGraphJob(String jobName, String outputPath) throws IOException {
+        PregelixJob job = new PregelixJob(jobName);
+        job.setVertexClass(SplitRepeatVertex.class);
+        job.setVertexInputFormatClass(InitialGraphCleanInputFormat.class);
+        job.setVertexOutputFormatClass(GraphCleanOutputFormat.class); 
+        job.setDynamicVertexValueSize(true);
+        job.setOutputKeyClass(VKmerBytesWritable.class);
+        job.setOutputValueClass(VertexValueWritable.class);
+        job.getConfiguration().setInt(SplitRepeatVertex.KMER_SIZE, 3);
+        job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+    }
+
+    private static void genSplitRepeatGraph() throws IOException {
+        generateSplitRepeatGraphJob("SplitRepeatGraph", outputBase + "SplitRepeatGraph.xml");
+    }
+    
+    
     private static void generateBridgeAddGraphJob(String jobName, String outputPath) throws IOException {
         PregelixJob job = new PregelixJob(jobName);
         job.setVertexClass(BridgeAddVertex.class);
@@ -228,10 +249,28 @@
                 + "BubbleMergeGraph.xml");
     }
     
+    private static void generateScaffoldingGraphJob(String jobName, String outputPath) throws IOException {
+        PregelixJob job = new PregelixJob(jobName);
+        job.setVertexClass(ScaffoldingVertex.class);
+        job.setVertexInputFormatClass(GraphCleanInputFormat.class);
+        job.setVertexOutputFormatClass(GraphCleanOutputFormat.class);
+        job.setDynamicVertexValueSize(true);
+        job.setOutputKeyClass(VKmerBytesWritable.class);
+        job.setOutputValueClass(VertexValueWritable.class);
+        job.getConfiguration().setInt(ScaffoldingVertex.KMER_SIZE, 3);
+        job.getConfiguration().writeXml(new FileOutputStream(new File(outputPath)));
+    }
+
+    private static void genScaffoldingGraph() throws IOException {
+        generateScaffoldingGraphJob("ScaffoldingGraph", outputBase
+                + "ScaffoldingGraph.xml");
+    }
+    
     public static void main(String[] args) throws IOException {
         genMapReduceGraph();
         genLogAlgorithmForMergeGraph();
         genP4ForMergeGraph();
+        genRemoveLowCoverageGraph();
         genTipAddGraph();
         genTipRemoveGraph();
         genBridgeAddGraph();
@@ -239,6 +278,7 @@
         genBubbleAddGraph();
         genBubbleMergeGraph();
         genSplitRepeatGraph();
+        genScaffoldingGraph();
     }
 
 }
diff --git a/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobRun/PathMergeSmallTestSuite.java b/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobRun/PathMergeSmallTestSuite.java
index b1b0fc0..7b937c8 100644
--- a/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobRun/PathMergeSmallTestSuite.java
+++ b/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobRun/PathMergeSmallTestSuite.java
@@ -45,14 +45,14 @@
 
     public static final String PreFix = "data/PathMergeTestSet"; //"graphbuildresult";
     public static final String[] TestDir = { PreFix + File.separator
-//    + "2", PreFix + File.separator
-//    + "3", PreFix + File.separator
-//    + "4", PreFix + File.separator
-//    + "5", PreFix + File.separator
-//    + "6", PreFix + File.separator
-//    + "7", PreFix + File.separator
-//    + "8", PreFix + File.separator
-    + "2"};
+    + "2", PreFix + File.separator
+    + "3", PreFix + File.separator
+    + "4", PreFix + File.separator
+    + "5", PreFix + File.separator
+    + "6", PreFix + File.separator
+    + "7", PreFix + File.separator
+    + "8", PreFix + File.separator
+    + "9"};
     private static final String ACTUAL_RESULT_DIR = "data/actual/pathmerge";
     private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
     private static final String PATH_TO_CLUSTER_STORE = "src/test/resources/cluster/stores.properties";
diff --git a/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobRun/RemoveLowCoverageSmallTestSuite.java b/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobRun/RemoveLowCoverageSmallTestSuite.java
new file mode 100644
index 0000000..9a7d99e
--- /dev/null
+++ b/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobRun/RemoveLowCoverageSmallTestSuite.java
@@ -0,0 +1,201 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * 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 from
+ *
+ *     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 edu.uci.ics.genomix.pregelix.JobRun;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+
+import junit.framework.Test;
+import junit.framework.TestResult;
+import junit.framework.TestSuite;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+
+@SuppressWarnings("deprecation")
+public class RemoveLowCoverageSmallTestSuite extends TestSuite {
+    private static final Logger LOGGER = Logger.getLogger(RemoveLowCoverageSmallTestSuite.class.getName());
+
+    public static final String PreFix = "data/PathMergeTestSet"; //"graphbuildresult";
+    public static final String[] TestDir = { PreFix + File.separator
+    + "5"};
+    private static final String ACTUAL_RESULT_DIR = "data/actual/removelowcoverage";
+    private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+    private static final String PATH_TO_CLUSTER_STORE = "src/test/resources/cluster/stores.properties";
+    private static final String PATH_TO_CLUSTER_PROPERTIES = "src/test/resources/cluster/cluster.properties";
+    private static final String PATH_TO_JOBS = "src/test/resources/jobs/";
+    private static final String PATH_TO_ONLY = "src/test/resources/only_removelowcoverage.txt";
+
+    public static final String HDFS_INPUTPATH = "/PathTestSet";
+
+    private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
+    private MiniDFSCluster dfsCluster;
+
+    private JobConf conf = new JobConf();
+    private int numberOfNC = 2;
+
+    public void setUp() throws Exception {
+        ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
+        ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
+        cleanupStores();
+        PregelixHyracksIntegrationUtil.init("src/test/resources/topology.xml");
+        LOGGER.info("Hyracks mini-cluster started");
+        FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+        FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+        startHDFS();
+    }
+
+    private void startHDFS() throws IOException {
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+        FileSystem lfs = FileSystem.getLocal(new Configuration());
+        lfs.delete(new Path("build"), true);
+        System.setProperty("hadoop.log.dir", "logs");
+        dfsCluster = new MiniDFSCluster(conf, numberOfNC, true, null);
+        FileSystem dfs = FileSystem.get(conf);
+
+        for (String testDir : TestDir) {
+            File src = new File(testDir);
+            Path dest = new Path(HDFS_INPUTPATH + File.separator + src.getName());
+            dfs.mkdirs(dest);
+            //src.listFiles()
+            //src.listFiles((FilenameFilter)(new WildcardFileFilter("part*")))
+            for (File f : src.listFiles()) {
+                dfs.copyFromLocalFile(new Path(f.getAbsolutePath()), dest);
+            }
+        }
+
+        DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
+        conf.writeXml(confOutput);
+        confOutput.flush();
+        confOutput.close();
+    }
+
+    private void cleanupStores() throws IOException {
+        FileUtils.forceMkdir(new File("teststore"));
+        FileUtils.forceMkdir(new File("build"));
+        FileUtils.cleanDirectory(new File("teststore"));
+        FileUtils.cleanDirectory(new File("build"));
+    }
+
+    /**
+     * cleanup hdfs cluster
+     */
+    private void cleanupHDFS() throws Exception {
+        dfsCluster.shutdown();
+    }
+
+    public void tearDown() throws Exception {
+        PregelixHyracksIntegrationUtil.deinit();
+        LOGGER.info("Hyracks mini-cluster shut down");
+        cleanupHDFS();
+    }
+
+    public static Test suite() throws Exception {
+        List<String> onlys = getFileList(PATH_TO_ONLY);
+        File testData = new File(PATH_TO_JOBS);
+        File[] queries = testData.listFiles();
+        RemoveLowCoverageSmallTestSuite testSuite = new RemoveLowCoverageSmallTestSuite();
+        testSuite.setUp();
+        boolean onlyEnabled = false;
+        FileSystem dfs = FileSystem.get(testSuite.conf);
+
+        if (onlys.size() > 0) {
+            onlyEnabled = true;
+        }
+
+        for (File qFile : queries) {
+            if (qFile.isFile()) {
+                if (onlyEnabled && !isInList(onlys, qFile.getName())) {
+                    continue;
+                } else {
+                    for (String testPathStr : TestDir) {
+                        File testDir = new File(testPathStr);
+                        String resultFileName = ACTUAL_RESULT_DIR + File.separator + jobExtToResExt(qFile.getName())
+                                + File.separator + "bin" + File.separator + testDir.getName();
+                        String textFileName = ACTUAL_RESULT_DIR + File.separator + jobExtToResExt(qFile.getName())
+                                + File.separator + "txt" + File.separator + testDir.getName();
+                        String graphvizFileName = ACTUAL_RESULT_DIR + File.separator + jobExtToResExt(qFile.getName())
+                                + File.separator + "graphviz" + File.separator + testDir.getName();
+                        testSuite.addTest(new BasicSmallTestCase(HADOOP_CONF_PATH, qFile.getName(), qFile
+                                .getAbsolutePath().toString(), dfs,
+                                HDFS_INPUTPATH + File.separator + testDir.getName(), resultFileName, textFileName, graphvizFileName));
+                    }
+                }
+            }
+        }
+        return testSuite;
+    }
+
+    /**
+     * Runs the tests and collects their result in a TestResult.
+     */
+    @Override
+    public void run(TestResult result) {
+        try {
+            int testCount = countTestCases();
+            for (int i = 0; i < testCount; i++) {
+                // cleanupStores();
+                Test each = this.testAt(i);
+                if (result.shouldStop())
+                    break;
+                runTest(each, result);
+            }
+            tearDown();
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    protected static List<String> getFileList(String ignorePath) throws FileNotFoundException, IOException {
+        BufferedReader reader = new BufferedReader(new FileReader(ignorePath));
+        String s = null;
+        List<String> ignores = new ArrayList<String>();
+        while ((s = reader.readLine()) != null) {
+            ignores.add(s);
+        }
+        reader.close();
+        return ignores;
+    }
+
+    private static String jobExtToResExt(String fname) {
+        int dot = fname.lastIndexOf('.');
+        return fname.substring(0, dot);
+    }
+
+    private static boolean isInList(List<String> onlys, String name) {
+        for (String only : onlys)
+            if (name.indexOf(only) >= 0)
+                return true;
+        return false;
+    }
+
+}
diff --git a/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobRun/ScaffoldingSmallTestSuite.java b/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobRun/ScaffoldingSmallTestSuite.java
new file mode 100644
index 0000000..c8d12dc
--- /dev/null
+++ b/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/JobRun/ScaffoldingSmallTestSuite.java
@@ -0,0 +1,201 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * 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 from
+ *
+ *     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 edu.uci.ics.genomix.pregelix.JobRun;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+
+import junit.framework.Test;
+import junit.framework.TestResult;
+import junit.framework.TestSuite;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+
+@SuppressWarnings("deprecation")
+public class ScaffoldingSmallTestSuite extends TestSuite {
+    private static final Logger LOGGER = Logger.getLogger(ScaffoldingSmallTestSuite.class.getName());
+    //P4ForMergeGraph/bin/read
+    public static final String PreFix = "data/actual/pathmerge/P4ForMergeGraph/bin"; 
+    public static final String[] TestDir = { PreFix + File.separator
+    + "2"};
+    private static final String ACTUAL_RESULT_DIR = "data/actual/scaffolding";
+    private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+    private static final String PATH_TO_CLUSTER_STORE = "src/test/resources/cluster/stores.properties";
+    private static final String PATH_TO_CLUSTER_PROPERTIES = "src/test/resources/cluster/cluster.properties";
+    private static final String PATH_TO_JOBS = "src/test/resources/jobs/";
+    private static final String PATH_TO_ONLY = "src/test/resources/only_scaffolding.txt";
+
+    public static final String HDFS_INPUTPATH = "/PathTestSet";
+
+    private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
+    private MiniDFSCluster dfsCluster;
+
+    private JobConf conf = new JobConf();
+    private int numberOfNC = 1;
+
+    public void setUp() throws Exception {
+        ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
+        ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
+        cleanupStores();
+        PregelixHyracksIntegrationUtil.init("src/test/resources/topology.xml");
+        LOGGER.info("Hyracks mini-cluster started");
+        FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+        FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+        startHDFS();
+    }
+
+    private void startHDFS() throws IOException {
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+        FileSystem lfs = FileSystem.getLocal(new Configuration());
+        lfs.delete(new Path("build"), true);
+        System.setProperty("hadoop.log.dir", "logs");
+        dfsCluster = new MiniDFSCluster(conf, numberOfNC, true, null);
+        FileSystem dfs = FileSystem.get(conf);
+
+        for (String testDir : TestDir) {
+            File src = new File(testDir);
+            Path dest = new Path(HDFS_INPUTPATH + File.separator + src.getName());
+            dfs.mkdirs(dest);
+            //src.listFiles()
+            //src.listFiles((FilenameFilter)(new WildcardFileFilter("part*")))
+            for (File f : src.listFiles()) {
+                dfs.copyFromLocalFile(new Path(f.getAbsolutePath()), dest);
+            }
+        }
+
+        DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
+        conf.writeXml(confOutput);
+        confOutput.flush();
+        confOutput.close();
+    }
+
+    private void cleanupStores() throws IOException {
+        FileUtils.forceMkdir(new File("teststore"));
+        FileUtils.forceMkdir(new File("build"));
+        FileUtils.cleanDirectory(new File("teststore"));
+        FileUtils.cleanDirectory(new File("build"));
+    }
+
+    /**
+     * cleanup hdfs cluster
+     */
+    private void cleanupHDFS() throws Exception {
+        dfsCluster.shutdown();
+    }
+
+    public void tearDown() throws Exception {
+        PregelixHyracksIntegrationUtil.deinit();
+        LOGGER.info("Hyracks mini-cluster shut down");
+        cleanupHDFS();
+    }
+
+    public static Test suite() throws Exception {
+        List<String> onlys = getFileList(PATH_TO_ONLY);
+        File testData = new File(PATH_TO_JOBS);
+        File[] queries = testData.listFiles();
+        ScaffoldingSmallTestSuite testSuite = new ScaffoldingSmallTestSuite();
+        testSuite.setUp();
+        boolean onlyEnabled = false;
+        FileSystem dfs = FileSystem.get(testSuite.conf);
+
+        if (onlys.size() > 0) {
+            onlyEnabled = true;
+        }
+
+        for (File qFile : queries) {
+            if (qFile.isFile()) {
+                if (onlyEnabled && !isInList(onlys, qFile.getName())) {
+                    continue;
+                } else {
+                    for (String testPathStr : TestDir) {
+                        File testDir = new File(testPathStr);
+                        String resultFileName = ACTUAL_RESULT_DIR + File.separator + jobExtToResExt(qFile.getName())
+                                + File.separator + "bin" + File.separator + testDir.getName();
+                        String textFileName = ACTUAL_RESULT_DIR + File.separator + jobExtToResExt(qFile.getName())
+                                + File.separator + "txt" + File.separator + testDir.getName();
+                        String graphvizFileName = ACTUAL_RESULT_DIR + File.separator + jobExtToResExt(qFile.getName())
+                                + File.separator + "graphviz" + File.separator + testDir.getName();
+                        testSuite.addTest(new BasicSmallTestCase(HADOOP_CONF_PATH, qFile.getName(), qFile
+                                .getAbsolutePath().toString(), dfs,
+                                HDFS_INPUTPATH + File.separator + testDir.getName(), resultFileName, textFileName, graphvizFileName));
+                    }
+                }
+            }
+        }
+        return testSuite;
+    }
+
+    /**
+     * Runs the tests and collects their result in a TestResult.
+     */
+    @Override
+    public void run(TestResult result) {
+        try {
+            int testCount = countTestCases();
+            for (int i = 0; i < testCount; i++) {
+                // cleanupStores();
+                Test each = this.testAt(i);
+                if (result.shouldStop())
+                    break;
+                runTest(each, result);
+            }
+            tearDown();
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    protected static List<String> getFileList(String ignorePath) throws FileNotFoundException, IOException {
+        BufferedReader reader = new BufferedReader(new FileReader(ignorePath));
+        String s = null;
+        List<String> ignores = new ArrayList<String>();
+        while ((s = reader.readLine()) != null) {
+            ignores.add(s);
+        }
+        reader.close();
+        return ignores;
+    }
+
+    private static String jobExtToResExt(String fname) {
+        int dot = fname.lastIndexOf('.');
+        return fname.substring(0, dot);
+    }
+
+    private static boolean isInList(List<String> onlys, String name) {
+        for (String only : onlys)
+            if (name.indexOf(only) >= 0)
+                return true;
+        return false;
+    }
+
+}
diff --git a/genomix/genomix-pregelix/src/test/resources/only_pathmerge.txt b/genomix/genomix-pregelix/src/test/resources/only_pathmerge.txt
index 5a15ca0..3d007d2 100644
--- a/genomix/genomix-pregelix/src/test/resources/only_pathmerge.txt
+++ b/genomix/genomix-pregelix/src/test/resources/only_pathmerge.txt
@@ -1 +1 @@
-LogAlgorithmForMergeGraph.xml
+P4ForMergeGraph.xml
diff --git a/genomix/genomix-pregelix/src/test/resources/only_removelowcoverage.txt b/genomix/genomix-pregelix/src/test/resources/only_removelowcoverage.txt
new file mode 100644
index 0000000..77ff8f0
--- /dev/null
+++ b/genomix/genomix-pregelix/src/test/resources/only_removelowcoverage.txt
@@ -0,0 +1 @@
+RemoveLowCoverageGraph.xml
diff --git a/genomix/genomix-pregelix/src/test/resources/only_scaffolding.txt b/genomix/genomix-pregelix/src/test/resources/only_scaffolding.txt
new file mode 100644
index 0000000..fed8efd
--- /dev/null
+++ b/genomix/genomix-pregelix/src/test/resources/only_scaffolding.txt
@@ -0,0 +1 @@
+ScaffoldingGraph.xml