finished EndToEnd test
diff --git a/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/KmerBytesWritable.java b/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/KmerBytesWritable.java
index 8f9094f..4883176 100644
--- a/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/KmerBytesWritable.java
+++ b/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/KmerBytesWritable.java
@@ -77,13 +77,13 @@
         if (newData == null) {
             this.set(0, EMPTY_BYTES, 0);
         } else {
-            this.set(newData.kmerlength, newData.bytes, 0);
+            this.set(newData.kmerlength, newData.bytes, newData.getOffset());
         }
     }
 
     public void set(byte[] newData, int offset) {
         if (kmerlength > 0) {
-            System.arraycopy(newData, offset, bytes, offset, size);
+            System.arraycopy(newData, offset, bytes, this.offset, size);
         }
     }
 
diff --git a/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/NodeWritable.java b/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/NodeWritable.java
index 7f78b68..0276c3a 100644
--- a/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/NodeWritable.java
+++ b/genomix/genomix-data/src/main/java/edu/uci/ics/genomix/type/NodeWritable.java
@@ -13,14 +13,12 @@
      */
     private static final long serialVersionUID = 1L;
     private PositionWritable nodeID;
-    private int countOfKmer;
     private PositionListWritable incomingList;
     private PositionListWritable outgoingList;
     private KmerBytesWritable kmer;
     
     public NodeWritable(){
         nodeID = new PositionWritable();
-        countOfKmer = 0;
         incomingList = new PositionListWritable();
         outgoingList = new PositionListWritable();
         kmer = new KmerBytesWritable();
@@ -28,18 +26,13 @@
 
     public NodeWritable(int kmerSize) {
         nodeID = new PositionWritable();
-        countOfKmer = 0;
         incomingList = new PositionListWritable();
         outgoingList = new PositionListWritable();
         kmer = new KmerBytesWritable(kmerSize);
     }
 
     public int getCount() {
-        return countOfKmer;
-    }
-
-    public void setCount(int count) {
-        this.countOfKmer = count;
+        return kmer.getKmerLength();
     }
 
     public void setNodeID(PositionWritable ref) {
@@ -58,11 +51,11 @@
         outgoingList.set(outgoing);
     }
 
-    public void reset() {
+    public void reset(int kmerSize) {
         nodeID.set(0, (byte) 0);
         incomingList.reset();
         outgoingList.reset();
-        countOfKmer = 0;
+        kmer.reset(kmerSize);
     }
 
     public PositionListWritable getIncomingList() {
@@ -82,14 +75,12 @@
     }
 
     public void mergeNextWithinOneRead(NodeWritable nextNodeEntry) {
-        this.countOfKmer += 1;
         this.outgoingList.set(nextNodeEntry.outgoingList);
-        kmer.mergeKmerWithNextCode(nextNodeEntry.kmer.getGeneCodeAtPosition(nextNodeEntry.kmer.getKmerLength() - 1));
+        kmer.mergeKmerWithNextCode(nextNodeEntry.kmer.getGeneCodeAtPosition(nextNodeEntry.getCount() - 1));
     }
 
     public void set(NodeWritable node) {
         this.nodeID.set(node.getNodeID().getReadID(), node.getNodeID().getPosInRead());
-        this.countOfKmer = node.countOfKmer;
         this.incomingList.set(node.getIncomingList());
         this.outgoingList.set(node.getOutgoingList());
         this.kmer.set(node.kmer);
@@ -98,7 +89,6 @@
     @Override
     public void readFields(DataInput in) throws IOException {
         this.nodeID.readFields(in);
-        this.countOfKmer = in.readInt();
         this.incomingList.readFields(in);
         this.outgoingList.readFields(in);
         this.kmer.readFields(in);
@@ -107,7 +97,6 @@
     @Override
     public void write(DataOutput out) throws IOException {
         this.nodeID.write(out);
-        out.writeInt(this.countOfKmer);
         this.incomingList.write(out);
         this.outgoingList.write(out);
         this.kmer.write(out);
@@ -127,10 +116,9 @@
     public String toString(){
         StringBuilder sbuilder = new StringBuilder();
         sbuilder.append('(');
-        sbuilder.append(nodeID.toString()).append(',');
-        sbuilder.append(countOfKmer).append(',');
-        sbuilder.append(incomingList.toString()).append(',');
-        sbuilder.append(incomingList.toString()).append(',');
+        sbuilder.append(nodeID.toString()).append('\t');
+        sbuilder.append(incomingList.toString()).append('\t');
+        sbuilder.append(outgoingList.toString()).append('\t');
         sbuilder.append(kmer.toString()).append(')');
         return sbuilder.toString();
     }
diff --git a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/MapReadToNodeOperator.java b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/MapReadToNodeOperator.java
index 46a92ec..d27c73d 100644
--- a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/MapReadToNodeOperator.java
+++ b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/MapReadToNodeOperator.java
@@ -4,12 +4,12 @@
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.genomix.hyracks.data.primitive.NodeReference;
-import edu.uci.ics.genomix.hyracks.data.primitive.PositionReference;
 import edu.uci.ics.genomix.type.KmerBytesWritable;
 import edu.uci.ics.genomix.type.PositionListWritable;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
@@ -43,6 +43,9 @@
     public static final int OutputOutgoingField = 3;
     public static final int OutputKmerBytesField = 4;
 
+    public static final RecordDescriptor nodeOutputRec = new RecordDescriptor(new ISerializerDeserializer[] { null,
+            null, null, null, null });
+
     /**
      * (ReadID, Storage[posInRead]={len, PositionList, len, Kmer})
      * to (Position, LengthCount, InComingPosList, OutgoingPosList, Kmer)
@@ -62,7 +65,7 @@
         private NodeReference nextNodeEntry;
 
         public MapReadToNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc,
-                RecordDescriptor outputRecDesc, int kmerSize) {
+                RecordDescriptor outputRecDesc) {
             this.ctx = ctx;
             this.inputRecDesc = inputRecDesc;
             this.outputRecDesc = outputRecDesc;
@@ -78,7 +81,7 @@
             appender = new FrameTupleAppender(ctx.getFrameSize());
             appender.reset(writeBuffer, true);
             writer.open();
-            curNodeEntry.reset();
+            curNodeEntry.reset(kmerSize);
         }
 
         @Override
@@ -113,6 +116,44 @@
             outputNode(curNodeEntry);
         }
 
+        private void resetNode(NodeReference node, int readID, byte posInRead, int offset, boolean byRef) {
+            node.reset(kmerSize);
+            node.setNodeID(readID, posInRead);
+
+            ByteBuffer buffer = accessor.getBuffer();
+            int lengthPos = buffer.getInt(offset);
+            int countPosition = PositionListWritable.getCountByDataLength(lengthPos);
+            offset += INT_LENGTH;
+            if (posInRead == 0) {
+                setPositionList(node.getIncomingList(), countPosition, buffer.array(), offset, byRef);
+            } else {
+                setPositionList(node.getOutgoingList(), countPosition, buffer.array(), offset, byRef);
+            }
+            offset += lengthPos;
+            int lengthKmer = buffer.getInt(offset);
+            if (node.getKmer().getLength() != lengthKmer) {
+                throw new IllegalStateException("Size of Kmer is invalid ");
+            }
+            setKmer(node.getKmer(), buffer.array(), offset + INT_LENGTH, byRef);
+        }
+
+        private void setKmer(KmerBytesWritable kmer, byte[] array, int offset, boolean byRef) {
+            if (byRef) {
+                kmer.setNewReference(array, offset);
+            } else {
+                kmer.set(array, offset);
+            }
+        }
+
+        private void setPositionList(PositionListWritable positionListWritable, int count, byte[] array, int offset,
+                boolean byRef) {
+            if (byRef) {
+                positionListWritable.setNewReference(count, array, offset);
+            } else {
+                positionListWritable.set(count, array, offset);
+            }
+        }
+
         private void outputNode(NodeReference node) throws HyracksDataException {
             try {
                 builder.addField(node.getNodeID().getByteArray(), node.getNodeID().getStartOffset(), node.getNodeID()
@@ -138,48 +179,6 @@
             }
         }
 
-        private void resetNode(NodeReference node, int readID, byte posInRead, int offset, boolean byRef) {
-            node.reset();
-            node.setNodeID(readID, posInRead);
-
-            ByteBuffer buffer = accessor.getBuffer();
-            int lengthOfPosition = buffer.getInt(offset);
-            if (lengthOfPosition % PositionReference.LENGTH != 0) {
-                throw new IllegalStateException("Size of PositionList is invalid ");
-            }
-            offset += INT_LENGTH;
-            if (posInRead == 0) {
-                setPositionList(node.getIncomingList(), lengthOfPosition / PositionReference.LENGTH, buffer.array(),
-                        offset, byRef);
-            } else {
-                setPositionList(node.getOutgoingList(), lengthOfPosition / PositionReference.LENGTH, buffer.array(),
-                        offset, byRef);
-            }
-            offset += lengthOfPosition;
-            int lengthKmer = buffer.getInt(offset);
-            if (node.getKmer().getLength() != lengthKmer) {
-                throw new IllegalStateException("Size of Kmer is invalid ");
-            }
-            setKmer(node.getKmer(), buffer.array(), offset + INT_LENGTH, byRef);
-            node.setCount(1);
-        }
-
-        private void setKmer(KmerBytesWritable kmer, byte[] array, int offset, boolean byRef) {
-            if (byRef) {
-                kmer.setNewReference(array, offset);
-            } else {
-                kmer.set(array, offset);
-            }
-        }
-
-        private void setPositionList(PositionListWritable positionListWritable, int count, byte[] array, int offset, boolean byRef) {
-            if (byRef) {
-                positionListWritable.setNewReference(count, array, offset);
-            } else {
-                positionListWritable.set(count, array, offset);
-            }
-        }
-
         @Override
         public void fail() throws HyracksDataException {
             writer.fail();
@@ -200,7 +199,7 @@
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         // TODO Auto-generated method stub
         return new MapReadToNodePushable(ctx, recordDescProvider.getInputRecordDescriptor(getActivityId(), 0),
-                recordDescriptors[0], kmerSize);
+                recordDescriptors[0]);
     }
 
 }
diff --git a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/io/NodeSequenceWriterFactory.java b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/io/NodeSequenceWriterFactory.java
index 684d01e..df4c08b 100644
--- a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/io/NodeSequenceWriterFactory.java
+++ b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/io/NodeSequenceWriterFactory.java
@@ -67,15 +67,15 @@
         public void write(DataOutput output, ITupleReference tuple) throws HyracksDataException {
             node.getNodeID().setNewReference(tuple.getFieldData(InputNodeIDField),
                     tuple.getFieldStart(InputNodeIDField));
-            node.setCount(Marshal.getInt(tuple.getFieldData(InputCountOfKmerField),
-                    tuple.getFieldStart(InputCountOfKmerField)));
             node.getIncomingList().setNewReference(tuple.getFieldLength(InputIncomingField) / PositionWritable.LENGTH,
                     tuple.getFieldData(InputIncomingField), tuple.getFieldStart(InputIncomingField));
             node.getOutgoingList().setNewReference(tuple.getFieldLength(InputOutgoingField) / PositionWritable.LENGTH,
                     tuple.getFieldData(InputOutgoingField), tuple.getFieldStart(InputOutgoingField));
 
-            node.getKmer().setNewReference(node.getCount() + kmerlength - 1, tuple.getFieldData(InputKmerBytesField),
-                    tuple.getFieldStart(InputKmerBytesField));
+            node.getKmer().setNewReference(
+                    Marshal.getInt(tuple.getFieldData(NodeSequenceWriterFactory.InputCountOfKmerField),
+                            tuple.getFieldStart(NodeSequenceWriterFactory.InputCountOfKmerField)),
+                    tuple.getFieldData(InputKmerBytesField), tuple.getFieldStart(InputKmerBytesField));
 
             try {
                 writer.append(node, null);
diff --git a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/io/NodeTextWriterFactory.java b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/io/NodeTextWriterFactory.java
index 5b549b5..c24760f 100644
--- a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/io/NodeTextWriterFactory.java
+++ b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/dataflow/io/NodeTextWriterFactory.java
@@ -5,7 +5,7 @@
 
 import edu.uci.ics.genomix.data.Marshal;
 import edu.uci.ics.genomix.type.NodeWritable;
-import edu.uci.ics.genomix.type.PositionWritable;
+import edu.uci.ics.genomix.type.PositionListWritable;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -39,22 +39,25 @@
             public void write(DataOutput output, ITupleReference tuple) throws HyracksDataException {
                 node.getNodeID().setNewReference(tuple.getFieldData(NodeSequenceWriterFactory.InputNodeIDField),
                         tuple.getFieldStart(NodeSequenceWriterFactory.InputNodeIDField));
-                node.setCount(Marshal.getInt(tuple.getFieldData(NodeSequenceWriterFactory.InputCountOfKmerField),
-                        tuple.getFieldStart(NodeSequenceWriterFactory.InputCountOfKmerField)));
                 node.getIncomingList().setNewReference(
-                        tuple.getFieldLength(NodeSequenceWriterFactory.InputIncomingField) / PositionWritable.LENGTH,
+                        PositionListWritable.getCountByDataLength(tuple
+                                .getFieldLength(NodeSequenceWriterFactory.InputIncomingField)),
                         tuple.getFieldData(NodeSequenceWriterFactory.InputIncomingField),
                         tuple.getFieldStart(NodeSequenceWriterFactory.InputIncomingField));
                 node.getOutgoingList().setNewReference(
-                        tuple.getFieldLength(NodeSequenceWriterFactory.InputOutgoingField) / PositionWritable.LENGTH,
+                        PositionListWritable.getCountByDataLength(tuple
+                                .getFieldLength(NodeSequenceWriterFactory.InputOutgoingField)),
                         tuple.getFieldData(NodeSequenceWriterFactory.InputOutgoingField),
                         tuple.getFieldStart(NodeSequenceWriterFactory.InputOutgoingField));
 
-                node.getKmer().setNewReference(node.getCount() + initialKmerSize - 1,
+                node.getKmer().setNewReference(
+                        Marshal.getInt(tuple.getFieldData(NodeSequenceWriterFactory.InputCountOfKmerField),
+                                tuple.getFieldStart(NodeSequenceWriterFactory.InputCountOfKmerField)),
                         tuple.getFieldData(NodeSequenceWriterFactory.InputKmerBytesField),
                         tuple.getFieldStart(NodeSequenceWriterFactory.InputKmerBytesField));
                 try {
                     output.write(node.toString().getBytes());
+                    output.writeByte('\n');
                 } catch (IOException e) {
                     throw new HyracksDataException(e);
                 }
diff --git a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/job/JobGenBrujinGraph.java b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/job/JobGenBrujinGraph.java
index bc160cd..02d9f9c 100644
--- a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/job/JobGenBrujinGraph.java
+++ b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/job/JobGenBrujinGraph.java
@@ -257,8 +257,9 @@
             AbstractOperatorDescriptor readCrossAggregator) {
         // Map (ReadID, [(Poslist,Kmer) ... ]) to (Node, IncomingList,
         // OutgoingList, Kmer)
-        RecordDescriptor nodeRec = new RecordDescriptor(new ISerializerDeserializer[] { null, null, null, null });
-        AbstractOperatorDescriptor mapEachReadToNode = new MapReadToNodeOperator(jobSpec, nodeRec, kmerSize);
+
+        AbstractOperatorDescriptor mapEachReadToNode = new MapReadToNodeOperator(jobSpec,
+                MapReadToNodeOperator.nodeOutputRec, kmerSize);
         connectOperators(jobSpec, readCrossAggregator, ncNodeNames, mapEachReadToNode, ncNodeNames,
                 new OneToOneConnectorDescriptor(jobSpec));
         return mapEachReadToNode;
@@ -327,6 +328,8 @@
         logDebug("Group by Read Operator");
         lastOperator = generateGroupbyReadJob(jobSpec, lastOperator);
 
+        logDebug("Generate final node");
+        lastOperator = generateMapperFromReadToNode(jobSpec, lastOperator);
         logDebug("Write node to result");
         lastOperator = generateNodeWriterOpertator(jobSpec, lastOperator);
 
diff --git a/genomix/genomix-hyracks/src/test/java/edu/uci/ics/genomix/hyracks/test/JobRunStepByStepTest.java b/genomix/genomix-hyracks/src/test/java/edu/uci/ics/genomix/hyracks/test/JobRunStepByStepTest.java
index 631fc5b..ca9ada0 100644
--- a/genomix/genomix-hyracks/src/test/java/edu/uci/ics/genomix/hyracks/test/JobRunStepByStepTest.java
+++ b/genomix/genomix-hyracks/src/test/java/edu/uci/ics/genomix/hyracks/test/JobRunStepByStepTest.java
@@ -44,6 +44,7 @@
     private static final String EXPECTED_OUPUT_KMER = EXPECTED_DIR + "result_after_kmerAggregate";
     private static final String EXPECTED_KMER_TO_READID = EXPECTED_DIR + "result_after_kmer2readId";
     private static final String EXPECTED_GROUPBYREADID = EXPECTED_DIR + "result_after_readIDAggreage";
+    private static final String EXPECTED_OUPUT_NODE = EXPECTED_DIR + "result_after_generateNode";
 
     private static final String DUMPED_RESULT = ACTUAL_RESULT_DIR + HDFS_OUTPUT_PATH + "/merged.txt";
     private static final String CONVERT_RESULT = DUMPED_RESULT + ".txt";
@@ -58,18 +59,18 @@
 
     @Test
     public void TestAll() throws Exception {
-        //TestReader();
-        //TestGroupbyKmer();
-        //TestMapKmerToRead();
+        TestReader();
+        TestGroupbyKmer();
+        TestMapKmerToRead();
         TestGroupByReadID();
-        // TestEndToEnd();
+        TestEndToEnd();
     }
 
     public void TestReader() throws Exception {
         cleanUpReEntry();
         conf.set(GenomixJobConf.OUTPUT_FORMAT, GenomixJobConf.OUTPUT_FORMAT_TEXT);
         driver.runJob(new GenomixJobConf(conf), Plan.CHECK_KMERREADER, true);
-        Assert.assertEquals(true, checkResults(EXPECTED_READER_RESULT, -1));
+        Assert.assertEquals(true, checkResults(EXPECTED_READER_RESULT, null));
     }
 
     public void TestGroupbyKmer() throws Exception {
@@ -77,14 +78,14 @@
         conf.set(GenomixJobConf.OUTPUT_FORMAT, GenomixJobConf.OUTPUT_FORMAT_TEXT);
         conf.set(GenomixJobConf.GROUPBY_TYPE, GenomixJobConf.GROUPBY_TYPE_PRECLUSTER);
         driver.runJob(new GenomixJobConf(conf), Plan.OUTPUT_KMERHASHTABLE, true);
-        Assert.assertEquals(true, checkResults(EXPECTED_OUPUT_KMER, 1));
+        Assert.assertEquals(true, checkResults(EXPECTED_OUPUT_KMER, new int[] { 1 }));
     }
 
     public void TestMapKmerToRead() throws Exception {
         cleanUpReEntry();
         conf.set(GenomixJobConf.OUTPUT_FORMAT, GenomixJobConf.OUTPUT_FORMAT_TEXT);
         driver.runJob(new GenomixJobConf(conf), Plan.OUTPUT_MAP_KMER_TO_READ, true);
-        Assert.assertEquals(true, checkResults(EXPECTED_KMER_TO_READID, 2));
+        Assert.assertEquals(true, checkResults(EXPECTED_KMER_TO_READID, new int[] { 2 }));
     }
 
     public void TestGroupByReadID() throws Exception {
@@ -92,19 +93,15 @@
         conf.set(GenomixJobConf.OUTPUT_FORMAT, GenomixJobConf.OUTPUT_FORMAT_TEXT);
         conf.set(GenomixJobConf.GROUPBY_TYPE, GenomixJobConf.GROUPBY_TYPE_PRECLUSTER);
         driver.runJob(new GenomixJobConf(conf), Plan.OUTPUT_GROUPBY_READID, true);
-        Assert.assertEquals(true, checkResults(EXPECTED_GROUPBYREADID, -1));
+        Assert.assertEquals(true, checkResults(EXPECTED_GROUPBYREADID, null));
     }
 
     public void TestEndToEnd() throws Exception {
         conf.set(GenomixJobConf.OUTPUT_FORMAT, GenomixJobConf.OUTPUT_FORMAT_TEXT);
         cleanUpReEntry();
-        conf.set(GenomixJobConf.GROUPBY_TYPE, GenomixJobConf.GROUPBY_TYPE_EXTERNAL);
-        driver.runJob(new GenomixJobConf(conf), Plan.BUILD_DEBRUJIN_GRAPH, true);
-        Assert.assertEquals(true, checkResults(EXPECTED_OUPUT_KMER, -1));
-        cleanUpReEntry();
         conf.set(GenomixJobConf.GROUPBY_TYPE, GenomixJobConf.GROUPBY_TYPE_PRECLUSTER);
         driver.runJob(new GenomixJobConf(conf), Plan.BUILD_DEBRUJIN_GRAPH, true);
-        Assert.assertEquals(true, checkResults(EXPECTED_OUPUT_KMER, -1));
+        Assert.assertEquals(true, checkResults(EXPECTED_OUPUT_NODE, new int[] {1,2}));
     }
 
     @Before
@@ -164,7 +161,7 @@
         }
     }
 
-    private boolean checkResults(String expectedPath, int poslistField) throws Exception {
+    private boolean checkResults(String expectedPath, int[] poslistField) throws Exception {
         File dumped = null;
         String format = conf.get(GenomixJobConf.OUTPUT_FORMAT);
         if (GenomixJobConf.OUTPUT_FORMAT_TEXT.equalsIgnoreCase(format)) {
@@ -210,7 +207,7 @@
             dumped = new File(CONVERT_RESULT);
         }
 
-        if (poslistField > 0) {
+        if (poslistField != null) {
             TestUtils.compareWithUnSortedPosition(new File(expectedPath), dumped, poslistField);
         } else {
             TestUtils.compareWithSortedResult(new File(expectedPath), dumped);
diff --git a/genomix/genomix-hyracks/src/test/java/edu/uci/ics/genomix/hyracks/test/TestUtils.java b/genomix/genomix-hyracks/src/test/java/edu/uci/ics/genomix/hyracks/test/TestUtils.java
index 49e21ba..6e6a504 100644
--- a/genomix/genomix-hyracks/src/test/java/edu/uci/ics/genomix/hyracks/test/TestUtils.java
+++ b/genomix/genomix-hyracks/src/test/java/edu/uci/ics/genomix/hyracks/test/TestUtils.java
@@ -61,7 +61,7 @@
         }
     }
 
-    public static void compareWithUnSortedPosition(File expectedFile, File actualFile, int poslistField)
+    public static void compareWithUnSortedPosition(File expectedFile, File actualFile, int[] poslistField)
             throws Exception {
         BufferedReader readerActual = new BufferedReader(new FileReader(actualFile));
         BufferedReader readerExpected = new BufferedReader(new FileReader(expectedFile));
@@ -162,15 +162,22 @@
         }
         return true;
     }
-
-    private static boolean containStrings(String lineExpected, String actualLine, int poslistField) {
+    
+    private static boolean containStrings(String lineExpected, String actualLine, int[] poslistField) {
         String[] fieldsExp = lineExpected.split("\\\t");
         String[] fieldsAct = actualLine.split("\\\t");
         if (fieldsAct.length != fieldsExp.length) {
             return false;
         }
         for (int i = 0; i < fieldsAct.length; i++) {
-            if (i == poslistField) {
+            boolean cont = false;
+            for (int x : poslistField) {
+                if (i == x) {
+                    cont = true;
+                    break;
+                }
+            }
+            if (cont){
                 continue;
             }
             if (!fieldsAct[i].equals(fieldsExp[i])) {
@@ -181,29 +188,44 @@
         ArrayList<String> posExp = new ArrayList<String>();
         ArrayList<String> posAct = new ArrayList<String>();
 
-        String valueExp = lineExpected.split("\\\t")[poslistField];
-        String[] valuesExp = valueExp.substring(1, valueExp.length() - 1).split(",");
+        for (int x : poslistField) {
+            String valueExp = lineExpected.split("\\\t")[x];
+            for (int i = 1; i < valueExp.length() - 1;) {
+                if (valueExp.charAt(i) == '(') {
+                    String str = "";
+                    i++;
+                    while (i < valueExp.length() - 1 && valueExp.charAt(i) != ')') {
+                        str += valueExp.charAt(i);
+                        i++;
+                    }
+                    posExp.add(str);
+                }
+                i++;
+            }
+            String valueAct = actualLine.split("\\\t")[x];
+            for (int i = 1; i < valueAct.length() - 1;) {
+                if (valueAct.charAt(i) == '(') {
+                    String str = "";
+                    i++;
+                    while (i < valueAct.length() - 1 && valueAct.charAt(i) != ')') {
+                        str += valueAct.charAt(i);
+                        i++;
+                    }
+                    posAct.add(str);
+                }
+                i++;
+            }
 
-        for (String str : valuesExp) {
-            posExp.add(str);
-        }
-
-        String valueAct = actualLine.split("\\\t")[poslistField];
-        String[] valuesAct = valueAct.substring(1, valueAct.length() - 1).split(",");
-
-        for (String str : valuesAct) {
-            posAct.add(str);
-        }
-
-        if (posExp.size() != posAct.size()) {
-            return false;
-        }
-        Collections.sort(posExp);
-        Collections.sort(posAct);
-        for (int i = 0; i < posExp.size(); i++) {
-            if (!posExp.get(i).equals(posAct.get(i))) {
+            if (posExp.size() != posAct.size()) {
                 return false;
             }
+            Collections.sort(posExp);
+            Collections.sort(posAct);
+            for (int i = 0; i < posExp.size(); i++) {
+                if (!posExp.get(i).equals(posAct.get(i))) {
+                    return false;
+                }
+            }
         }
         return true;
     }
diff --git a/genomix/genomix-hyracks/src/test/resources/expected/result_after_generateNode b/genomix/genomix-hyracks/src/test/resources/expected/result_after_generateNode
new file mode 100644
index 0000000..01d4a27
--- /dev/null
+++ b/genomix/genomix-hyracks/src/test/resources/expected/result_after_generateNode
@@ -0,0 +1,12 @@
+((1,0)	[]	[(6,0),(1,3)]	AATAGAA)
+((1,3)	[(1,0)]	[(6,0)]	AGAAG)
+((2,0)	[]	[(6,0),(2,3)]	AATAGAA)
+((2,3)	[(2,0)]	[(6,0)]	AGAAG)
+((3,0)	[]	[(6,0),(3,3)]	AATAGAA)
+((3,3)	[(3,0)]	[(6,0)]	AGAAG)
+((4,0)	[]	[(6,0),(4,3)]	AATAGAA)
+((4,3)	[(4,0)]	[(6,0)]	AGAAG)
+((5,0)	[]	[(6,0),(5,3)]	AATAGAA)
+((5,3)	[(5,0)]	[(6,0)]	AGAAG)
+((6,0)	[(1,3),(2,3),(3,3),(5,3),(6,3),(4,3)]	[(6,0),(6,3)]	AGAAGAA)
+((6,3)	[(6,0)]	[(6,0)]	AGAAG)