Merge branch 'genomix/fullstack_genomix' into wbiesing/refactor_p4

use static final values "debug" and "problemKmers" to log all actions
for given set of kmers

Conflicts:
	genomix/genomix-
pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/P4ForPathMergeVertex.java
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
index 9955a63..3f1aae2 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HashPartitionMergeExchangePOperator.java
@@ -44,9 +44,11 @@
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -142,27 +144,32 @@
         int n = orderColumns.size();
         int[] sortFields = new int[n];
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[n];
-        {
-            int j = 0;
-            for (OrderColumn oc : orderColumns) {
-                LogicalVariable var = oc.getColumn();
-                sortFields[j] = opSchema.findVariable(var);
-                Object type = env.getVarType(var);
-                IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
-                comparatorFactories[j] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
-                j++;
+
+        INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+        INormalizedKeyComputerFactory nkcf = null;
+
+        int j = 0;
+        for (OrderColumn oc : orderColumns) {
+            LogicalVariable var = oc.getColumn();
+            sortFields[j] = opSchema.findVariable(var);
+            Object type = env.getVarType(var);
+            IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+            comparatorFactories[j] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
+            if (j == 0 && nkcfProvider != null && type != null) {
+                nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, oc.getOrder() == OrderKind.ASC);
             }
+            j++;
         }
 
         IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields,
-                comparatorFactories);
+                comparatorFactories, nkcf);
         return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
     }
-    
+
     public List<LogicalVariable> getPartitionFields() {
         return partitionFields;
     }
-    
+
     public List<OrderColumn> getOrderColumns() {
         return orderColumns;
     }
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
index dc345ac..b03b99d 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
@@ -39,9 +39,11 @@
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
+import edu.uci.ics.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -128,6 +130,10 @@
         IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
         IBinaryHashFunctionFactory[] hashFuns = new IBinaryHashFunctionFactory[n];
         IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+
+        INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+        INormalizedKeyComputerFactory nkcf = null;
+
         for (int i = 0; i < n; i++) {
             sortFields[i] = opSchema.findVariable(sortColumns[i].getColumn());
             Object type = env.getVarType(sortColumns[i].getColumn());
@@ -135,9 +141,12 @@
             comps[i] = bcfp.getBinaryComparatorFactory(type, sortColumns[i].getOrder() == OrderKind.ASC);
             IBinaryHashFunctionFactoryProvider bhffp = context.getBinaryHashFunctionFactoryProvider();
             hashFuns[i] = bhffp.getBinaryHashFunctionFactory(type);
+            if (i == 0 && nkcfProvider != null && type != null) {
+                nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, sortColumns[i].getOrder() == OrderKind.ASC);
+            }
         }
         ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(sortFields, hashFuns);
-        IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps);
+        IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps, nkcf);
         return new Pair<IConnectorDescriptor, TargetConstraint>(conn, TargetConstraint.ONE);
     }
 
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
index b299e78..131eea0 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
@@ -24,7 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorterMergeSort;
 
 public class InMemorySortRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
@@ -52,12 +52,12 @@
 
         return new AbstractOneInputOneOutputPushRuntime() {
 
-            FrameSorter frameSorter = null;
+            FrameSorterMergeSort frameSorter = null;
 
             @Override
             public void open() throws HyracksDataException {
                 if (frameSorter == null) {
-                    frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+                    frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
                             outputRecordDesc);
                 }
                 frameSorter.reset();
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 ffcb449..0250a73 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
@@ -21,8 +21,10 @@
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.AbstractMap;
 import java.util.Comparator;
 import java.util.EnumSet;
+import java.util.Map;
 
 import org.apache.hadoop.io.WritableComparable;
 
@@ -778,6 +780,21 @@
                 break;
         }
     }
+    
+    /**
+     * Debug helper function to find the edge associated with the given kmer.
+     * 
+     * Note: may be very slow-- does a linear scan of all edges!
+     */
+    public Map.Entry<EDGETYPE, EdgeWritable> findEdge(final VKmerBytesWritable kmer) {
+        for (EDGETYPE dir : EDGETYPE.values()) {
+            for (EdgeWritable e : edges[dir.get()]) {
+                if (e.getKey().equals(kmer))
+                    return new AbstractMap.SimpleEntry<EDGETYPE, EdgeWritable>(dir, e);
+            }
+        }
+        return null;
+    }
 
     public int inDegree() {
         return edges[EDGETYPE.RR.get()].getCountOfPosition() + edges[EDGETYPE.RF.get()].getCountOfPosition();
diff --git a/genomix/genomix-driver/src/main/java/edu/uci/ics/genomix/driver/GenomixDriver.java b/genomix/genomix-driver/src/main/java/edu/uci/ics/genomix/driver/GenomixDriver.java
index 488bc45..831cc17 100644
--- a/genomix/genomix-driver/src/main/java/edu/uci/ics/genomix/driver/GenomixDriver.java
+++ b/genomix/genomix-driver/src/main/java/edu/uci/ics/genomix/driver/GenomixDriver.java
@@ -58,6 +58,7 @@
  */
 public class GenomixDriver {
 
+    public static final Logger GENOMIX_ROOT_LOG = Logger.getLogger("edu.uci.ics.genomix");  // here only so we can control children loggers 
     private static final Logger LOG = Logger.getLogger(GenomixDriver.class.getName());
     private static final String HADOOP_CONF = "hadoop.conf.xml";
     private String prevOutput;
diff --git a/genomix/genomix-driver/src/main/resources/conf/logging.properties b/genomix/genomix-driver/src/main/resources/conf/logging.properties
index 5269ae1..bdb1318 100644
--- a/genomix/genomix-driver/src/main/resources/conf/logging.properties
+++ b/genomix/genomix-driver/src/main/resources/conf/logging.properties
@@ -20,8 +20,8 @@
 
 # To also add the FileHandler, use the following line instead.
 
-#handlers= java.util.logging.FileHandler, java.util.logging.ConsoleHandler
-handlers= java.util.logging.FileHandler
+handlers= java.util.logging.FileHandler, java.util.logging.ConsoleHandler
+#handlers= java.util.logging.FileHandler
 
 # Default global logging level.
 # This specifies which kinds of events are logged across
@@ -30,8 +30,8 @@
 # Note that the ConsoleHandler also has a separate level
 # setting to limit messages printed to the console.
 
-#.level= SEVERE
- .level= INFO
+.level= SEVERE
+# .level= INFO
 # .level= FINE
 # .level = FINEST
 
@@ -42,18 +42,18 @@
 
 # default file output is in user's home directory.
 
- java.util.logging.FileHandler.pattern = logs/genomix-pregelix-tests.log
+ java.util.logging.FileHandler.pattern = genomix-%u.log
  java.util.logging.FileHandler.limit = 0
  java.util.logging.FileHandler.count = 1
 # java.util.logging.FileHandler.formatter = java.util.logging.XMLFormatter
-# java.util.logging.FileHandler.formatter = java.util.logging.SimpleFormatter
- java.util.logging.FileHandler.formatter = edu.uci.ics.genomix.pregelix.log.PathMergeLogFormatter
+ java.util.logging.FileHandler.formatter = java.util.logging.SimpleFormatter
+# java.util.logging.FileHandler.formatter = edu.uci.ics.genomix.pregelix.log.PathMergeLogFormatter
  java.util.logging.FileHandler.level = FINE
 
 # Limit the message that are printed on the console to FINE and above.
 
-#java.util.logging.ConsoleHandler.level = INFO
-#java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter
+java.util.logging.ConsoleHandler.level = INFO
+java.util.logging.ConsoleHandler.formatter = java.util.logging.SimpleFormatter
 #java.util.logging.ConsoleHandler.formatter = java.util.logging.XMLFormatter
 
 
@@ -65,7 +65,8 @@
 # For example, set the com.xyz.foo logger to only log SEVERE
 # messages:
 
-edu.uci.ics.genomix.pregelix.level = FINE
+edu.uci.ics.genomix.level = ALL
+edu.uci.ics.pregelix.core.driver.Driver.level = INFO
 #edu.uci.ics.asterix.level = FINE
 #edu.uci.ics.algebricks.level = FINE
 #edu.uci.ics.hyracks.level = SEVERE
diff --git a/genomix/genomix-hadoop/src/test/python/getKmerNeighbors.py b/genomix/genomix-hadoop/src/test/python/getKmerNeighbors.py
index 4e73653..d84b026 100644
--- a/genomix/genomix-hadoop/src/test/python/getKmerNeighbors.py
+++ b/genomix/genomix-hadoop/src/test/python/getKmerNeighbors.py
@@ -3,24 +3,31 @@
 import sys
 import string
 import pickle
+from itertools import imap
 
 
-
-def getneighbors(readids, kmerlength, reads, index, skipreads):
-    "return a set of readids containing any of the kmers in the given readid"
+def getneighbors(ids_to_search, kmerlength, readindex, kmerindex, skipids):
+    """return a set of ids_to_search containing any of the kmers in the given readid
+    :param dict ids_to_search:
+    :param int kmerlength:
+    :param dict readindex:
+    :param dict kmerindex:
+    :param set skipids:
+    """
     neighbors = set()
-    for origreadid in readids:
-        print 'searching ', origreadid
-        seq = reads[origreadid]
-        for i in range(len(seq) - kmerlength):
-            subseq = seq[i:i+kmerlength]
-            neighbors.update(index.get(subseq, set()) - skipreads)
-            neighbors.update(index.get(revcomp(subseq), set()) - skipreads)
+    for rid in ids_to_search:
+        print 'searching ', rid
+        read = readindex[rid]
+        for i in range(len(read) - kmerlength):
+            kmer = read[i:i + kmerlength]
+            neighbors.update(kmerindex.get(kmer, set()) - skipids)
+            neighbors.update(kmerindex.get(revcomp(kmer), set()) - skipids)
     return neighbors
 
 
-_revcomp_table = string.maketrans('ACGT', 'TGCA')
-def revcomp(s):
+def revcomp(s, _revcomp_table=string.maketrans('ACGT', 'TGCA')):
+    """return the reverse-complement of s
+    """
     return string.translate(s, _revcomp_table)[::-1]
 
 
@@ -30,51 +37,40 @@
         infile = argv[2]
         numhops = int(argv[3])
         outname = argv[4]
-    except:
+    except IndexError:
         print "usage: <prog> origkmer infile numhops outname"
         return -1
 
     kmerlength = len(origkmer)
 
-    try:
-        print 'reading in', infile + ".reads.pkl"
-        reads = pickle.load(open(infile + ".reads.pkl"))
-    except:
-        print 'reading in', infile
-        reads = {line.strip().split('\t')[0] : line.strip().split('\t')[1] for line in open(infile)}
-        pickle.dump(reads, open(infile + ".reads.pkl", 'w'), -1)
+    print 'reading in', infile
+    readindex = {rid: read for rid, read in imap(lambda line: line.strip().split('\t'), open(infile))}
 
-    #try:
-    #    print 'reading', infile + '.index.pkl'
-    #    index = pickle.load(open(infile + ".index.pkl"))
-    #except:
-    index = {}
+    kmerindex = {}
     print 'building index'
-    for readid, line in reads.items():
-        for i in range(len(line) - kmerlength):
-            index.setdefault(line[i:i+kmerlength], set()).add(readid)
-    #pickle.dump(index, open(infile + ".index.pkl", 'w'), -1)
+    for rid, read in readindex.items():
+        for i in range(len(read) - kmerlength):
+            kmerindex.setdefault(read[i:i + kmerlength], set()).add(rid)
 
     outreads = set()
 
     print 'getting matching readids'
     curkmer = origkmer
-    outreads.update(index.get(curkmer, set()))
-    outreads.update(index.get(revcomp(curkmer), set()))
+    outreads.update(kmerindex.get(curkmer, set()))
+    outreads.update(kmerindex.get(revcomp(curkmer), set()))
 
     # get neighbors
     new_neighbors = outreads
     for i in range(numhops):
         print 'hop', i
-        new_neighbors = getneighbors(new_neighbors, kmerlength, reads, index, outreads)
+        new_neighbors = getneighbors(new_neighbors, kmerlength, readindex, kmerindex, outreads)
         outreads.update(new_neighbors)
 
     print 'found', len(outreads), 'neighbors'
 
     with open(outname, 'w') as outfile:
-        for readid in outreads:
-            outfile.write(readid + '\t' + reads[readid] + '\n')
-
+        for rid in outreads:
+            outfile.write(rid + '\t' + readindex[rid] + '\n')
 
 
 if __name__ == '__main__':
diff --git a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/graph/driver/Driver.java b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/graph/driver/Driver.java
index ae76da3..7dbfed3 100644
--- a/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/graph/driver/Driver.java
+++ b/genomix/genomix-hyracks/src/main/java/edu/uci/ics/genomix/hyracks/graph/driver/Driver.java
@@ -141,10 +141,10 @@
     private DeploymentId prepareJobs() throws Exception {
         URLClassLoader classLoader = (URLClassLoader) this.getClass().getClassLoader();
         List<String> jars = new ArrayList<String>();
+        LOG.info("Deploying jar files to NC's");
         URL[] urls = classLoader.getURLs();
         for (URL url : urls)
             if (url.toString().endsWith(".jar")) {
-                LOG.info(url.toString());
                 jars.add(new File(url.getPath()).toString());
             }
         DeploymentId deploymentId = hcc.deployBinary(jars);
diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/BasicGraphCleanVertex.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/BasicGraphCleanVertex.java
index 373fdde..dcae00a 100644
--- a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/BasicGraphCleanVertex.java
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/BasicGraphCleanVertex.java
@@ -1,8 +1,10 @@
 package edu.uci.ics.genomix.pregelix.operator;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Random;
 import java.util.logging.Logger;
 
@@ -71,6 +73,20 @@
     protected EDGETYPE incomingEdgeType; //SplitRepeat and BubbleMerge
     protected EDGETYPE outgoingEdgeType; //SplitRepeat and BubbleMerge
     
+    protected static final List<VKmerBytesWritable> problemKmers = Arrays.asList(
+//            new VKmerBytesWritable("CCCGGCCTCCAGCGTGGGATACGCGAAGATGCCGCCGTAGGTGAGAATCTGGTTC"),
+//            new VKmerBytesWritable("GCAGGAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+//            new VKmerBytesWritable("GAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+//            new VKmerBytesWritable("GAGCAGGAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+//            new VKmerBytesWritable("GGAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+//            new VKmerBytesWritable("AGGAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+//            new VKmerBytesWritable("GCGACGTGCAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+//            new VKmerBytesWritable("GTCAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"),
+            new VKmerBytesWritable("AACACCGAGGGCGTGCCGACGAACCCCGAGGCCTACGCGAAGATGCGCGGCTGGA")
+            );
+    protected static final boolean debug = false;
+    protected boolean verbose = false;
+    
     protected HashMapWritable<ByteWritable, VLongWritable> counters = new HashMapWritable<ByteWritable, VLongWritable>();
     /**
      * initiate kmerSize, maxIteration
@@ -81,6 +97,10 @@
         if (maxIteration < 0)
             maxIteration = Integer.parseInt(getContext().getConfiguration().get(GenomixJobConf.GRAPH_CLEAN_MAX_ITERATIONS));
         GenomixJobConf.setGlobalStaticConstants(getContext().getConfiguration());
+        
+        verbose = false;
+        for (VKmerBytesWritable problemKmer : problemKmers)
+            verbose |= debug && (getVertexValue().getNode().findEdge(problemKmer) != null || getVertexId().equals(problemKmer));
     }
     
     public boolean isHeadNode(){
diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/BasicPathMergeVertex.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/BasicPathMergeVertex.java
index bfd35ca..47b71a1 100644
--- a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/BasicPathMergeVertex.java
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/BasicPathMergeVertex.java
@@ -1,7 +1,11 @@
 package edu.uci.ics.genomix.pregelix.operator.pathmerge;
 
+import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.Iterator;
+import java.util.logging.Logger;
+
+import org.apache.commons.lang3.StringUtils;
 
 import edu.uci.ics.genomix.pregelix.io.VertexValueWritable;
 import edu.uci.ics.genomix.pregelix.io.VertexValueWritable.P4State;
@@ -19,6 +23,9 @@
 
 public abstract class BasicPathMergeVertex<V extends VertexValueWritable, M extends PathMergeMessageWritable> extends
 	BasicGraphCleanVertex<V, M>{
+    
+    private static final Logger LOG = Logger.getLogger(BasicPathMergeVertex.class.getName());
+    
     protected static final boolean isP1 = true;
     protected static final boolean isP2 = false;
     protected static final boolean isP4 = true;
@@ -448,10 +455,17 @@
         for (DIR dir : dirsToRestrict) {
             for (EDGETYPE et : dir.edgeType()) {
                 for (VKmerBytesWritable destId : vertex.getEdgeList(et).getKeys()) {
+                    if (debug)
+                        for (VKmerBytesWritable problemKmer : problemKmers)
+                            verbose |= destId.equals(problemKmer);
                     outgoingMsg.reset();
                     outgoingMsg.setFlag(et.mirror().dir().get());
                     
-//                    LOG.info("send restriction from " + getVertexId() + " to " + destId + " in my " + d + " and their " + DirectionFlag.mirrorEdge(d) + " (" + DirectionFlag.dirFromEdgeType(DirectionFlag.mirrorEdge(d)) + "); I am " + getVertexValue());
+                    if (verbose)
+                        LOG.fine("send restriction from " + getVertexId() + " to " + destId + 
+                                " in my " + et + " and their " + et.mirror() + 
+                                " (" + EDGETYPE.dir(et.mirror()) + 
+                                "); I am " + getVertexValue());
                     sendMsg(destId, outgoingMsg);
                 }
             }
@@ -465,10 +479,12 @@
         short restrictedDirs = getVertexValue().getState();  // the directions (NEXT/PREVIOUS) that I'm not allowed to merge in
         boolean updated = false;
         while (msgIterator.hasNext()) {
-//            LOG.info("before restriction " + getVertexId() + ": " + DIR.fromByte(restrictedDirs));
+            if (verbose)
+                LOG.fine("before restriction " + getVertexId() + ": " + DIR.fromByte(restrictedDirs));
             incomingMsg = msgIterator.next();
             restrictedDirs |= incomingMsg.getFlag();
-//            LOG.info("after restriction " + getVertexId() + ": " + DIR.fromByte(restrictedDirs));
+            if (verbose)
+                LOG.fine("after restriction " + getVertexId() + ": " + DIR.fromByte(restrictedDirs));
             updated = true;
         }
         if (updated) {
@@ -506,10 +522,14 @@
                 EDGETYPE newEdgetype = EDGETYPE.resolveLinkThroughMiddleNode(updateEdge, mergeEdge);
                 outgoingMsg.getNode().setEdgeList(newEdgetype, getVertexValue().getEdgeList(mergeEdge));  // copy into outgoingMsg
             }
+            if (debug)
+                for (VKmerBytesWritable problemKmer : problemKmers)
+                    verbose |= outgoingMsg.getNode().findEdge(problemKmer) != null;
             
             // send the update to all kmers in this list // TODO perhaps we could skip all this if there are no neighbors here
             for (VKmerBytesWritable dest : vertex.getEdgeList(updateEdge).getKeys()) {
-//                LOG.info("send update message from " + getVertexId() + " to " + dest + ": " + outgoingMsg);
+                if (verbose)
+                    LOG.fine("send update message from " + getVertexId() + " to " + dest + ": " + outgoingMsg);
                 sendMsg(dest, outgoingMsg);
             }
         }
@@ -519,9 +539,14 @@
         VertexValueWritable vertex = getVertexValue(); 
         NodeWritable node = vertex.getNode();
         boolean updated = false;
+        ArrayList<PathMergeMessageWritable> allSeenMsgs = new ArrayList<PathMergeMessageWritable>();
         while (msgIterator.hasNext()) {
-//            LOG.info("before update from neighbor: " + getVertexValue());
             incomingMsg = msgIterator.next();
+            if (debug)
+                for (VKmerBytesWritable problemKmer : problemKmers)
+                    verbose |= incomingMsg.getNode().findEdge(problemKmer) != null || incomingMsg.getSourceVertexId().equals(problemKmer);
+            if (verbose)
+                LOG.fine("before update from neighbor: " + getVertexValue());
             // remove the edge to the node that will merge elsewhere
             node.getEdgeList(EDGETYPE.fromByte(incomingMsg.getFlag())).remove(incomingMsg.getSourceVertexId());
             // add the node this neighbor will merge into
@@ -529,8 +554,13 @@
                 node.getEdgeList(edgeType).unionUpdate(incomingMsg.getEdgeList(edgeType));
             }
             updated = true;
-//            LOG.info("after update from neighbor: " + getVertexValue());
+            if (verbose) { 
+                LOG.fine("after update from neighbor: " + getVertexValue());
+                allSeenMsgs.add(incomingMsg);
+            }
         }
+        if (verbose)
+            LOG.fine("All recieved updates:  \n{\n" + StringUtils.join(allSeenMsgs, "\n") + "\n}\n");
         if (updated) {
             if (DIR.enumSetFromByte(vertex.getState()).containsAll(EnumSet.allOf(DIR.class)))
                 voteToHalt();
@@ -554,11 +584,13 @@
             if (vertex.getDegree(mergeEdgetype.dir()) != 1)
                 throw new IllegalStateException("Merge attempted in node with degree in " + mergeEdgetype + " direction != 1!\n" + vertex);
             VKmerBytesWritable dest = vertex.getEdgeList(mergeEdgetype).get(0).getKey();
-//            LOG.info("send merge mesage from " + getVertexId() + " to " + dest + ": " + outgoingMsg + "; my restrictions are: " + DIR.enumSetFromByte(vertex.getState()) + ", their restrictions are: " + DIR.enumSetFromByte(outgoingMsg.getFlag()));
             sendMsg(dest, outgoingMsg);
-            
-//            LOG.info("killing self: " + getVertexId());
             deleteVertex(getVertexId());
+            
+            if (verbose) {
+                LOG.fine("send merge mesage from " + getVertexId() + " to " + dest + ": " + outgoingMsg + "; my restrictions are: " + DIR.enumSetFromByte(vertex.getState()) + ", their restrictions are: " + DIR.enumSetFromByte(outgoingMsg.getFlag()));
+                LOG.fine("killing self: " + getVertexId());
+            }
         }
     }
     
@@ -574,20 +606,27 @@
         @SuppressWarnings("unused")
         int numMerged = 0;
         while (msgIterator.hasNext()) {
-//            LOG.info("before merge: " + getVertexValue() + " restrictions: " + DIR.enumSetFromByte(state));
             incomingMsg = msgIterator.next();
+            if (debug)
+                for (VKmerBytesWritable problemKmer : problemKmers)
+                    verbose |= incomingMsg.getNode().findEdge(problemKmer) != null;
+            if (verbose)
+                LOG.fine("before merge: " + getVertexValue() + " restrictions: " + DIR.enumSetFromByte(state));
             senderEdgetype = EDGETYPE.fromByte(incomingMsg.getFlag());
             node.mergeWithNode(senderEdgetype, incomingMsg.getNode());
             state |= (byte) (incomingMsg.getFlag() & DIR.MASK);  // update incoming restricted directions
             numMerged++;
             updated = true;
-//            LOG.info("after merge: " + getVertexValue() + " restrictions: " + DIR.enumSetFromByte(state));
+            if (verbose)
+                LOG.fine("after merge: " + getVertexValue() + " restrictions: " + DIR.enumSetFromByte(state));
         }
         if(isTandemRepeat(getVertexValue())) {
             // tandem repeats can't merge anymore; restrict all future merges
             state |= DIR.NEXT.get();
             state |= DIR.PREVIOUS.get();
             updated = true;
+            if (verbose)
+                LOG.fine("recieveMerges is a tandem repeat: " + getVertexId() + " " + getVertexValue());
 //          updateStatisticsCounter(StatisticsCounter.Num_Cycles); 
         }
 //      updateStatisticsCounter(StatisticsCounter.Num_MergedNodes);
diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/P4ForPathMergeVertex.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/P4ForPathMergeVertex.java
index dba03fe..824d0da 100644
--- a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/P4ForPathMergeVertex.java
+++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/pathmerge/P4ForPathMergeVertex.java
@@ -1,8 +1,13 @@
 package edu.uci.ics.genomix.pregelix.operator.pathmerge;
 
+import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.Iterator;
 import java.util.Random;
+import java.util.logging.Logger;
+
+import org.apache.commons.lang3.StringUtils;
+import org.jfree.util.Log;
 
 import edu.uci.ics.genomix.config.GenomixJobConf;
 import edu.uci.ics.genomix.pregelix.client.Client;
@@ -13,6 +18,7 @@
 import edu.uci.ics.genomix.type.NodeWritable.DIR;
 import edu.uci.ics.genomix.type.NodeWritable.EDGETYPE;
 import edu.uci.ics.genomix.type.VKmerBytesWritable;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * Graph clean pattern: P4(Smart-algorithm) for path merge 
@@ -22,7 +28,7 @@
 public class P4ForPathMergeVertex extends
     BasicPathMergeVertex<VertexValueWritable, PathMergeMessageWritable> {
     
-//    private static final Logger LOG = Logger.getLogger(P4ForPathMergeVertex.class.getName());
+    private static final Logger LOG = Logger.getLogger(P4ForPathMergeVertex.class.getName());
     
     private static long randSeed = 1; //static for save memory
     private float probBeingRandomHead = -1;
@@ -76,14 +82,15 @@
         counters.clear();
         getVertexValue().getCounters().clear();
     }
-    
+
     protected boolean isNodeRandomHead(VKmerBytesWritable nodeKmer) {
         // "deterministically random", based on node id
         randGenerator.setSeed((randSeed ^ nodeKmer.hashCode()) * 10000 * getSuperstep());//randSeed + nodeID.hashCode()
         for(int i = 0; i < 500; i++)
             randGenerator.nextFloat();
         boolean isHead = randGenerator.nextFloat() < probBeingRandomHead;
-//        LOG.info("randomHead: " + nodeKmer + "=" + isHead);
+        if (verbose)
+            LOG.fine("randomHead: " + nodeKmer + "=" + isHead);
         return isHead;
     }
     
@@ -157,15 +164,17 @@
                 }
             }
         }
-//        if ((getVertexValue().getState() & P4State.MERGE) == 0) {
-//            LOG.info("No merge for " + getVertexId());
-//        } else {
-//            LOG.info("Merge from " + getVertexId() + " towards " + (getVertexValue().getState() & DirectionFlag.DIR_MASK) + "; node is " + getVertexValue());
-//        }
+        if (verbose) {
+//            if ((getVertexValue().getState() & P4State.MERGE) == 0) {
+//                LOG.fine("No merge for " + getVertexId());
+//            } else {
+                LOG.fine("Merge from " + getVertexId() + " towards " + (EDGETYPE.fromByte(getVertexValue().getState())) + "; node is " + getVertexValue());
+//            }
+        }
     }
     
     @Override
-    public void compute(Iterator<PathMergeMessageWritable> msgIterator) {
+    public void compute(Iterator<PathMergeMessageWritable> msgIterator) throws HyracksDataException {
         initVertex();
             
         if (getSuperstep() == 1) {
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index 09eeab6..c72ced1 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -278,6 +278,9 @@
             removePendingThread(ct);
         }
         if (!exceptions.isEmpty()) {
+            for (Exception e : exceptions) {
+                e.printStackTrace();
+            }
             NodeControllerService ncs = joblet.getNodeController();
             ExceptionUtils.setNodeIds(exceptions, ncs.getId());
             ncs.getWorkQueue().schedule(new NotifyTaskFailureWork(ncs, this, exceptions));
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
index 04b91b2..31cd29b 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
@@ -43,6 +43,7 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
 
@@ -53,8 +54,8 @@
     private final MarshalledWritable<Configuration> config;
     private final IInputSplitProviderFactory factory;
 
-    public MapperOperatorDescriptor(IOperatorDescriptorRegistry spec, int jobId, MarshalledWritable<Configuration> config,
-            IInputSplitProviderFactory factory) throws HyracksDataException {
+    public MapperOperatorDescriptor(IOperatorDescriptorRegistry spec, int jobId,
+            MarshalledWritable<Configuration> config, IInputSplitProviderFactory factory) throws HyracksDataException {
         super(spec, 0, 1);
         this.jobId = jobId;
         this.config = config;
@@ -94,7 +95,7 @@
 
             public void initBlock(int blockId) throws HyracksDataException {
                 runGen = new ExternalSortRunGenerator(ctx, new int[] { 0 }, null, comparatorFactories,
-                        helper.getMapOutputRecordDescriptorWithoutExtraFields(), framesLimit);
+                        helper.getMapOutputRecordDescriptorWithoutExtraFields(), Algorithm.MERGE_SORT, framesLimit);
                 this.blockId = blockId;
             }
 
@@ -114,7 +115,8 @@
                     runGen.nextFrame(frame);
                     fta.reset(frame, true);
                     if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                        throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + frame.capacity() + ")");
+                        throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
+                                + frame.capacity() + ")");
                     }
                 }
             }
@@ -224,7 +226,7 @@
                     comparators[i] = comparatorFactories[i].createBinaryComparator();
                 }
                 ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, runGen.getFrameSorter(),
-                        runGen.getRuns(), new int[] { 0 }, comparators,
+                        runGen.getRuns(), new int[] { 0 }, comparators, null,
                         helper.getMapOutputRecordDescriptorWithoutExtraFields(), framesLimit, delegatingWriter);
                 merger.process();
             }
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
index 30ba3ec..3bb78f9 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
@@ -108,7 +108,7 @@
             runs.add(rfw.createReader());
         }
         RunFileWriter rfw = new RunFileWriter(outFile, ctx.getIOManager());
-        ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, null, runs, new int[] { 0 }, comparators,
+        ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, null, runs, new int[] { 0 }, comparators, null,
                 recordDescriptor, framesLimit, rfw);
         merger.process();
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
index 235b0d0..2dda9cc 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
@@ -21,6 +21,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
@@ -31,18 +32,21 @@
     private final int nSenders;
     private final int[] sortFields;
     private final IBinaryComparator[] comparators;
+    private final INormalizedKeyComputer nmkComputer;
     private final RecordDescriptor recordDescriptor;
     private final IPartitionBatchManager pbm;
 
     private RunMergingFrameReader merger;
 
     public SortMergeFrameReader(IHyracksTaskContext ctx, int maxConcurrentMerges, int nSenders, int[] sortFields,
-            IBinaryComparator[] comparators, RecordDescriptor recordDescriptor, IPartitionBatchManager pbm) {
+            IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer, RecordDescriptor recordDescriptor,
+            IPartitionBatchManager pbm) {
         this.ctx = ctx;
         this.maxConcurrentMerges = maxConcurrentMerges;
         this.nSenders = nSenders;
         this.sortFields = sortFields;
         this.comparators = comparators;
+        this.nmkComputer = nmkComputer;
         this.recordDescriptor = recordDescriptor;
         this.pbm = pbm;
     }
@@ -57,7 +61,7 @@
             List<IFrameReader> batch = new ArrayList<IFrameReader>();
             pbm.getNextBatch(batch, nSenders);
             merger = new RunMergingFrameReader(ctx, batch.toArray(new IFrameReader[nSenders]), inFrames, sortFields,
-                    comparators, recordDescriptor);
+                    comparators, nmkComputer, recordDescriptor);
         } else {
             // multi level merge.
             throw new HyracksDataException("Not yet supported");
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
index 32269bd..1cf402b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
@@ -23,6 +23,8 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -39,19 +41,23 @@
     private final ITuplePartitionComputerFactory tpcf;
     private final int[] sortFields;
     private final IBinaryComparatorFactory[] comparatorFactories;
+    private final INormalizedKeyComputerFactory nkcFactory;
     private final boolean stable;
 
-    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec, ITuplePartitionComputerFactory tpcf,
-            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories) {
-        this(spec, tpcf, sortFields, comparatorFactories, false);
+    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+            ITuplePartitionComputerFactory tpcf, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
+            INormalizedKeyComputerFactory nkcFactory) {
+        this(spec, tpcf, sortFields, comparatorFactories, nkcFactory, false);
     }
 
-    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec, ITuplePartitionComputerFactory tpcf,
-            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, boolean stable) {
+    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+            ITuplePartitionComputerFactory tpcf, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
+            INormalizedKeyComputerFactory nkcFactory, boolean stable) {
         super(spec);
         this.tpcf = tpcf;
         this.sortFields = sortFields;
         this.comparatorFactories = comparatorFactories;
+        this.nkcFactory = nkcFactory;
         this.stable = stable;
     }
 
@@ -71,9 +77,10 @@
         for (int i = 0; i < comparatorFactories.length; ++i) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
+        INormalizedKeyComputer nmkComputer = nkcFactory == null ? null : nkcFactory.createNormalizedKeyComputer();
         IPartitionBatchManager pbm = new NonDeterministicPartitionBatchManager(nProducerPartitions);
         IFrameReader sortMergeFrameReader = new SortMergeFrameReader(ctx, nProducerPartitions, nProducerPartitions,
-                sortFields, comparators, recordDesc, pbm);
+                sortFields, comparators, nmkComputer, recordDesc, pbm);
         BitSet expectedPartitions = new BitSet();
         expectedPartitions.set(0, nProducerPartitions);
         return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sortMergeFrameReader, pbm);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
index 21b3b60..bb3d0f3 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
@@ -25,6 +25,8 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -47,6 +49,7 @@
     private final Object stateId;
     private final int[] keyFields;
     private final IBinaryComparator[] comparators;
+    private final INormalizedKeyComputer nmkComputer;
     private final AggregateState aggregateState;
     private final ArrayTupleBuilder tupleBuilder;
     private final int[] storedKeys;
@@ -76,7 +79,7 @@
     private final FrameTupleAccessor outFrameAccessor;
 
     ExternalGroupMergeOperatorNodePushable(IHyracksTaskContext ctx, Object stateId,
-            IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
+            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory nmkFactory, int[] keyFields,
             IAggregatorDescriptorFactory mergerFactory, boolean isOutputSorted, int framesLimit,
             RecordDescriptor outRecordDescriptor) throws HyracksDataException {
         this.stateId = stateId;
@@ -85,6 +88,7 @@
         for (int i = 0; i < comparatorFactories.length; ++i) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
+        this.nmkComputer = nmkFactory == null ? null : nmkFactory.createNormalizedKeyComputer();
         int[] keyFieldsInPartialResults = new int[keyFields.length];
         for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
             keyFieldsInPartialResults[i] = i;
@@ -181,7 +185,7 @@
             FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
             Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
             ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), outRecordDescriptor,
-                    runNumber, comparator);
+                    runNumber, comparator, keyFields, nmkComputer);
             /**
              * current tuple index in each run
              */
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
index b914702..9296ffa 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
@@ -130,8 +130,8 @@
                 IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
                 throws HyracksDataException {
             return new ExternalGroupMergeOperatorNodePushable(ctx, new TaskId(new ActivityId(getOperatorId(),
-                    AGGREGATE_ACTIVITY_ID), partition), comparatorFactories, keyFields, mergerFactory, isOutputSorted,
-                    framesLimit, recordDescriptors[0]);
+                    AGGREGATE_ACTIVITY_ID), partition), comparatorFactories, firstNormalizerFactory, keyFields,
+                    mergerFactory, isOutputSorted, framesLimit, recordDescriptors[0]);
         }
 
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Algorithm.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Algorithm.java
new file mode 100644
index 0000000..5c96bd0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Algorithm.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2013 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.hyracks.dataflow.std.sort;
+
+public enum Algorithm {
+    QUICK_SORT,
+    MERGE_SORT
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index f80a82c..e1315e7 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -51,6 +52,15 @@
     private final IBinaryComparatorFactory[] comparatorFactories;
     private final int framesLimit;
 
+    private Algorithm alg = Algorithm.MERGE_SORT;
+
+    public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, Algorithm alg) {
+        this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
+        this.alg = alg;
+    }
+
     public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
             IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
         this(spec, framesLimit, sortFields, null, comparatorFactories, recordDescriptor);
@@ -86,7 +96,7 @@
 
     public static class SortTaskState extends AbstractStateObject {
         private List<IFrameReader> runs;
-        private FrameSorter frameSorter;
+        private IFrameSorter frameSorter;
 
         public SortTaskState() {
         }
@@ -122,7 +132,7 @@
                 @Override
                 public void open() throws HyracksDataException {
                     runGen = new ExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory,
-                            comparatorFactories, recordDescriptors[0], framesLimit);
+                            comparatorFactories, recordDescriptors[0], alg, framesLimit);
                     runGen.open();
                 }
 
@@ -166,14 +176,16 @@
                     SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
                             SORT_ACTIVITY_ID), partition));
                     List<IFrameReader> runs = state.runs;
-                    FrameSorter frameSorter = state.frameSorter;
+                    IFrameSorter frameSorter = state.frameSorter;
                     IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
                     for (int i = 0; i < comparatorFactories.length; ++i) {
                         comparators[i] = comparatorFactories[i].createBinaryComparator();
                     }
+                    INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
+                            : firstKeyNormalizerFactory.createNormalizedKeyComputer();
                     int necessaryFrames = Math.min(runs.size() + 2, framesLimit);
                     ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, frameSorter, runs, sortFields,
-                            comparators, recordDescriptors[0], necessaryFrames, writer);
+                            comparators, nmkComputer, recordDescriptors[0], necessaryFrames, writer);
                     merger.process();
                 }
             };
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index b149e30..3736fca 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -30,15 +30,21 @@
 
 public class ExternalSortRunGenerator implements IFrameWriter {
     private final IHyracksTaskContext ctx;
-    private final FrameSorter frameSorter;
+    private final IFrameSorter frameSorter;
     private final List<IFrameReader> runs;
     private final int maxSortFrames;
 
     public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDesc, int framesLimit) throws HyracksDataException {
+            RecordDescriptor recordDesc, Algorithm alg, int framesLimit) throws HyracksDataException {
         this.ctx = ctx;
-        frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc);
+        if (alg == Algorithm.MERGE_SORT) {
+            frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+                    recordDesc);
+        } else {
+            frameSorter = new FrameSorterQuickSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+                    recordDesc);
+        }
         runs = new LinkedList<IFrameReader>();
         maxSortFrames = framesLimit - 1;
     }
@@ -87,7 +93,7 @@
     public void fail() throws HyracksDataException {
     }
 
-    public FrameSorter getFrameSorter() {
+    public IFrameSorter getFrameSorter() {
         return frameSorter;
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index 510dfd6..eaf4162 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -23,6 +23,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -59,6 +60,7 @@
     private final List<IFrameReader> runs;
     private final int[] sortFields;
     private final IBinaryComparator[] comparators;
+    private final INormalizedKeyComputer nmkComputer;
     private final RecordDescriptor recordDesc;
     private final int framesLimit;
     private final IFrameWriter writer;
@@ -66,8 +68,8 @@
     private ByteBuffer outFrame;
     private FrameTupleAppender outFrameAppender;
 
-    private FrameSorter frameSorter; // Used in External sort, no replacement
-                                     // selection
+    private IFrameSorter frameSorter; // Used in External sort, no replacement
+                                      // selection
     private FrameTupleAccessor outFrameAccessor; // Used in External sort, with
                                                  // replacement selection
     private final int outputLimit; // Used in External sort, with replacement
@@ -76,14 +78,15 @@
                              // selection and limit on output size
 
     // Constructor for external sort, no replacement selection
-    public ExternalSortRunMerger(IHyracksTaskContext ctx, FrameSorter frameSorter, List<IFrameReader> runs,
-            int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc, int framesLimit,
-            IFrameWriter writer) {
+    public ExternalSortRunMerger(IHyracksTaskContext ctx, IFrameSorter frameSorter, List<IFrameReader> runs,
+            int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+            RecordDescriptor recordDesc, int framesLimit, IFrameWriter writer) {
         this.ctx = ctx;
         this.frameSorter = frameSorter;
         this.runs = new LinkedList<IFrameReader>(runs);
         this.sortFields = sortFields;
         this.comparators = comparators;
+        this.nmkComputer = nmkComputer;
         this.recordDesc = recordDesc;
         this.framesLimit = framesLimit;
         this.writer = writer;
@@ -92,11 +95,13 @@
 
     // Constructor for external sort with replacement selection
     public ExternalSortRunMerger(IHyracksTaskContext ctx, int outputLimit, List<IFrameReader> runs, int[] sortFields,
-            IBinaryComparator[] comparators, RecordDescriptor recordDesc, int framesLimit, IFrameWriter writer) {
+            IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc,
+            int framesLimit, IFrameWriter writer) {
         this.ctx = ctx;
         this.runs = new LinkedList<IFrameReader>(runs);
         this.sortFields = sortFields;
         this.comparators = comparators;
+        this.nmkComputer = nmkComputer;
         this.recordDesc = recordDesc;
         this.framesLimit = framesLimit;
         this.writer = writer;
@@ -162,7 +167,7 @@
 
     private void merge(IFrameWriter mergeResultWriter, IFrameReader[] runCursors) throws HyracksDataException {
         RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields, comparators,
-                recordDesc);
+                nmkComputer, recordDesc);
         merger.open();
         try {
             while (merger.nextFrame(outFrame)) {
@@ -263,7 +268,7 @@
                 runCursors[i] = runs.get(i);
             }
             RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields,
-                    comparators, recordDesc);
+                    comparators, nmkComputer, recordDesc);
             merger.open();
             try {
                 while (merger.nextFrame(outFrame)) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
similarity index 97%
rename from hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
rename to hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
index a6bb4e2..cc0f1ef 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
@@ -31,7 +31,7 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 
-public class FrameSorter {
+public class FrameSorterMergeSort implements IFrameSorter {
     private final IHyracksTaskContext ctx;
     private final int[] sortFields;
     private final INormalizedKeyComputer nkc;
@@ -50,7 +50,7 @@
     private int[] tPointersTemp;
     private int tupleCount;
 
-    public FrameSorter(IHyracksTaskContext ctx, int[] sortFields,
+    public FrameSorterMergeSort(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor) throws HyracksDataException {
         this.ctx = ctx;
@@ -69,15 +69,18 @@
         dataFrameCount = 0;
     }
 
+    @Override
     public void reset() {
         dataFrameCount = 0;
         tupleCount = 0;
     }
 
+    @Override
     public int getFrameCount() {
         return dataFrameCount;
     }
 
+    @Override
     public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
         ByteBuffer copyFrame;
         if (dataFrameCount == buffers.size()) {
@@ -90,6 +93,7 @@
         ++dataFrameCount;
     }
 
+    @Override
     public void sortFrames() {
         int nBuffers = dataFrameCount;
         tupleCount = 0;
@@ -123,6 +127,7 @@
         }
     }
 
+    @Override
     public void flushFrames(IFrameWriter writer) throws HyracksDataException {
         appender.reset(outFrame, true);
         for (int ptr = 0; ptr < tupleCount; ++ptr) {
@@ -241,6 +246,7 @@
         return 0;
     }
 
+    @Override
     public void close() {
         this.buffers.clear();
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
similarity index 71%
copy from hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
copy to hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
index a6bb4e2..083f4a7 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
@@ -29,9 +29,8 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 
-public class FrameSorter {
+public class FrameSorterQuickSort implements IFrameSorter {
     private final IHyracksTaskContext ctx;
     private final int[] sortFields;
     private final INormalizedKeyComputer nkc;
@@ -47,10 +46,9 @@
 
     private int dataFrameCount;
     private int[] tPointers;
-    private int[] tPointersTemp;
     private int tupleCount;
 
-    public FrameSorter(IHyracksTaskContext ctx, int[] sortFields,
+    public FrameSorterQuickSort(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor) throws HyracksDataException {
         this.ctx = ctx;
@@ -69,15 +67,18 @@
         dataFrameCount = 0;
     }
 
+    @Override
     public void reset() {
         dataFrameCount = 0;
         tupleCount = 0;
     }
 
+    @Override
     public int getFrameCount() {
         return dataFrameCount;
     }
 
+    @Override
     public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
         ByteBuffer copyFrame;
         if (dataFrameCount == buffers.size()) {
@@ -90,6 +91,7 @@
         ++dataFrameCount;
     }
 
+    @Override
     public void sortFrames() {
         int nBuffers = dataFrameCount;
         tupleCount = 0;
@@ -118,11 +120,11 @@
             }
         }
         if (tupleCount > 0) {
-            tPointersTemp = new int[tPointers.length];
-            sort(0, tupleCount);
+            sort(tPointers, 0, tupleCount);
         }
     }
 
+    @Override
     public void flushFrames(IFrameWriter writer) throws HyracksDataException {
         appender.reset(outFrame, true);
         for (int ptr = 0; ptr < tupleCount; ++ptr) {
@@ -145,73 +147,75 @@
         }
     }
 
-    private void sort(int offset, int length) {
-        int step = 1;
-        int len = length;
-        int end = offset + len;
-        /** bottom-up merge */
-        while (step < len) {
-            /** merge */
-            for (int i = offset; i < end; i += 2 * step) {
-                int next = i + step;
-                if (next < end) {
-                    merge(i, next, step, Math.min(step, end - next));
-                } else {
-                    System.arraycopy(tPointers, i * 4, tPointersTemp, i * 4, (end - i) * 4);
+    private void sort(int[] tPointers, int offset, int length) {
+        int m = offset + (length >> 1);
+        int mi = tPointers[m * 4];
+        int mj = tPointers[m * 4 + 1];
+        int mv = tPointers[m * 4 + 3];
+
+        int a = offset;
+        int b = a;
+        int c = offset + length - 1;
+        int d = c;
+        while (true) {
+            while (b <= c) {
+                int cmp = compare(tPointers, b, mi, mj, mv);
+                if (cmp > 0) {
+                    break;
                 }
+                if (cmp == 0) {
+                    swap(tPointers, a++, b);
+                }
+                ++b;
             }
-            /** prepare next phase merge */
-            step *= 2;
-            int[] tmp = tPointersTemp;
-            tPointersTemp = tPointers;
-            tPointers = tmp;
-        }
-    }
-
-    /** Merge two subarrays into one */
-    private void merge(int start1, int start2, int len1, int len2) {
-        int targetPos = start1;
-        int pos1 = start1;
-        int pos2 = start2;
-        int end1 = start1 + len1 - 1;
-        int end2 = start2 + len2 - 1;
-        while (pos1 <= end1 && pos2 <= end2) {
-            int cmp = compare(pos1, pos2);
-            if (cmp <= 0) {
-                copy(pos1, targetPos);
-                pos1++;
-            } else {
-                copy(pos2, targetPos);
-                pos2++;
+            while (c >= b) {
+                int cmp = compare(tPointers, c, mi, mj, mv);
+                if (cmp < 0) {
+                    break;
+                }
+                if (cmp == 0) {
+                    swap(tPointers, c, d--);
+                }
+                --c;
             }
-            targetPos++;
+            if (b > c)
+                break;
+            swap(tPointers, b++, c--);
         }
-        if (pos1 <= end1) {
-            int rest = end1 - pos1 + 1;
-            System.arraycopy(tPointers, pos1 * 4, tPointersTemp, targetPos * 4, rest * 4);
+
+        int s;
+        int n = offset + length;
+        s = Math.min(a - offset, b - a);
+        vecswap(tPointers, offset, b - s, s);
+        s = Math.min(d - c, n - d - 1);
+        vecswap(tPointers, b, n - s, s);
+
+        if ((s = b - a) > 1) {
+            sort(tPointers, offset, s);
         }
-        if (pos2 <= end2) {
-            int rest = end2 - pos2 + 1;
-            System.arraycopy(tPointers, pos2 * 4, tPointersTemp, targetPos * 4, rest * 4);
+        if ((s = d - c) > 1) {
+            sort(tPointers, n - s, s);
         }
     }
 
-    private void copy(int src, int dest) {
-        tPointersTemp[dest * 4] = tPointers[src * 4];
-        tPointersTemp[dest * 4 + 1] = tPointers[src * 4 + 1];
-        tPointersTemp[dest * 4 + 2] = tPointers[src * 4 + 2];
-        tPointersTemp[dest * 4 + 3] = tPointers[src * 4 + 3];
+    private void swap(int x[], int a, int b) {
+        for (int i = 0; i < 4; ++i) {
+            int t = x[a * 4 + i];
+            x[a * 4 + i] = x[b * 4 + i];
+            x[b * 4 + i] = t;
+        }
     }
 
-    private int compare(int tp1, int tp2) {
+    private void vecswap(int x[], int a, int b, int n) {
+        for (int i = 0; i < n; i++, a++, b++) {
+            swap(x, a, b);
+        }
+    }
+
+    private int compare(int[] tPointers, int tp1, int tp2i, int tp2j, int tp2v) {
         int i1 = tPointers[tp1 * 4];
         int j1 = tPointers[tp1 * 4 + 1];
         int v1 = tPointers[tp1 * 4 + 3];
-
-        int tp2i = tPointers[tp2 * 4];
-        int tp2j = tPointers[tp2 * 4 + 1];
-        int tp2v = tPointers[tp2 * 4 + 3];
-
         if (v1 != tp2v) {
             return ((((long) v1) & 0xffffffffL) < (((long) tp2v) & 0xffffffffL)) ? -1 : 1;
         }
@@ -225,12 +229,12 @@
         fta2.reset(buf2);
         for (int f = 0; f < comparators.length; ++f) {
             int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf1.array(), j1 + (fIdx - 1) * 4);
-            int f1End = IntSerDeUtils.getInt(buf1.array(), j1 + fIdx * 4);
+            int f1Start = fIdx == 0 ? 0 : buf1.getInt(j1 + (fIdx - 1) * 4);
+            int f1End = buf1.getInt(j1 + fIdx * 4);
             int s1 = j1 + fta1.getFieldSlotsLength() + f1Start;
             int l1 = f1End - f1Start;
-            int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf2.array(), j2 + (fIdx - 1) * 4);
-            int f2End = IntSerDeUtils.getInt(buf2.array(), j2 + fIdx * 4);
+            int f2Start = fIdx == 0 ? 0 : buf2.getInt(j2 + (fIdx - 1) * 4);
+            int f2End = buf2.getInt(j2 + fIdx * 4);
             int s2 = j2 + fta2.getFieldSlotsLength() + f2Start;
             int l2 = f2End - f2Start;
             int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
@@ -241,7 +245,8 @@
         return 0;
     }
 
+    @Override
     public void close() {
         this.buffers.clear();
     }
-}
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
new file mode 100644
index 0000000..6778852
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2013 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.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameSorter {
+
+    public void reset();
+
+    public int getFrameCount();
+
+    public void insertFrame(ByteBuffer buffer) throws HyracksDataException;
+
+    public void sortFrames();
+
+    public void flushFrames(IFrameWriter writer) throws HyracksDataException;
+
+    public void close();
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index 04c82af..6fa21b5 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -77,7 +77,7 @@
     }
 
     public static class SortTaskState extends AbstractStateObject {
-        private FrameSorter frameSorter;
+        private FrameSorterMergeSort frameSorter;
 
         public SortTaskState() {
         }
@@ -111,7 +111,7 @@
                 @Override
                 public void open() throws HyracksDataException {
                     state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition));
-                    state.frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory,
+                    state.frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory,
                             comparatorFactories, recordDescriptors[0]);
                     state.frameSorter.reset();
                 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
index bff2e21..ef1ae88 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -201,9 +202,11 @@
                         comparators[i] = comparatorFactories[i].createBinaryComparator();
                     }
 
+                    INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
+                            : firstKeyNormalizerFactory.createNormalizedKeyComputer();
                     int necessaryFrames = Math.min(runs.size() + 2, memSize);
                     ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, outputLimit, runs, sortFields,
-                            comparators, recordDescriptors[0], necessaryFrames, writer);
+                            comparators, nmkComputer, recordDescriptors[0], necessaryFrames, writer);
 
                     merger.processWithReplacementSelection();
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
index 16b3c12..00fbe9b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -35,6 +36,7 @@
     private final List<ByteBuffer> inFrames;
     private final int[] sortFields;
     private final IBinaryComparator[] comparators;
+    private final INormalizedKeyComputer nmkComputer;
     private final RecordDescriptor recordDesc;
     private final FrameTupleAppender outFrameAppender;
     private ReferencedPriorityQueue topTuples;
@@ -42,12 +44,14 @@
     private FrameTupleAccessor[] tupleAccessors;
 
     public RunMergingFrameReader(IHyracksTaskContext ctx, IFrameReader[] runCursors, List<ByteBuffer> inFrames,
-            int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc) {
+            int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+            RecordDescriptor recordDesc) {
         this.ctx = ctx;
         this.runCursors = runCursors;
         this.inFrames = inFrames;
         this.sortFields = sortFields;
         this.comparators = comparators;
+        this.nmkComputer = nmkComputer;
         this.recordDesc = recordDesc;
         outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
     }
@@ -56,7 +60,8 @@
     public void open() throws HyracksDataException {
         tupleAccessors = new FrameTupleAccessor[runCursors.length];
         Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
-        topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, runCursors.length, comparator);
+        topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, runCursors.length, comparator,
+                sortFields, nmkComputer);
         tupleIndexes = new int[runCursors.length];
         for (int i = 0; i < runCursors.length; i++) {
             tupleIndexes[i] = 0;
@@ -143,21 +148,25 @@
     private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
         return new Comparator<ReferenceEntry>() {
             public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
+                int nmk1 = tp1.getNormalizedKey();
+                int nmk2 = tp1.getNormalizedKey();
+                if (nmk1 > nmk2) {
+                    return 1;
+                }
+                if (nmk1 < nmk2) {
+                    return -1;
+                }
+
                 FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
                 FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
-                int j1 = tp1.getTupleIndex();
-                int j2 = tp2.getTupleIndex();
                 byte[] b1 = fta1.getBuffer().array();
                 byte[] b2 = fta2.getBuffer().array();
+                int[] tPointers1 = tp1.getTPointers();
+                int[] tPointers2 = tp2.getTPointers();
+
                 for (int f = 0; f < sortFields.length; ++f) {
-                    int fIdx = sortFields[f];
-                    int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
-                            + fta1.getFieldStartOffset(j1, fIdx);
-                    int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
-                    int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
-                            + fta2.getFieldStartOffset(j2, fIdx);
-                    int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
-                    int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                    int c = comparators[f].compare(b1, tPointers1[2 * f + 1], tPointers1[2 * f + 2], b2,
+                            tPointers2[2 * f + 1], tPointers2[2 * f + 2]);
                     if (c != 0) {
                         return c;
                     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
index c1efa91..c06b50c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
@@ -14,18 +14,24 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.util;
 
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public class ReferenceEntry {
     private final int runid;
     private FrameTupleAccessor acccessor;
     private int tupleIndex;
+    private int[] tPointers;
 
-    public ReferenceEntry(int runid, FrameTupleAccessor fta, int tupleIndex) {
+    public ReferenceEntry(int runid, FrameTupleAccessor fta, int tupleIndex, int[] keyFields,
+            INormalizedKeyComputer nmkComputer) {
         super();
         this.runid = runid;
         this.acccessor = fta;
-        this.tupleIndex = tupleIndex;
+        this.tPointers = new int[1 + 2 * keyFields.length];
+        if (fta != null) {
+            initTPointer(fta, tupleIndex, keyFields, nmkComputer);
+        }
     }
 
     public int getRunid() {
@@ -40,11 +46,38 @@
         this.acccessor = fta;
     }
 
+    public int[] getTPointers() {
+        return tPointers;
+    }
+
     public int getTupleIndex() {
         return tupleIndex;
     }
 
-    public void setTupleIndex(int tupleIndex) {
+    public int getNormalizedKey() {
+        return tPointers[0];
+    }
+
+    public void setTupleIndex(int tupleIndex, int[] keyFields, INormalizedKeyComputer nmkComputer) {
+        initTPointer(acccessor, tupleIndex, keyFields, nmkComputer);
+    }
+
+    private void initTPointer(FrameTupleAccessor fta, int tupleIndex, int[] keyFields,
+            INormalizedKeyComputer nmkComputer) {
         this.tupleIndex = tupleIndex;
+        byte[] b1 = fta.getBuffer().array();
+        for (int f = 0; f < keyFields.length; ++f) {
+            int fIdx = keyFields[f];
+            tPointers[2 * f + 1] = fta.getTupleStartOffset(tupleIndex) + fta.getFieldSlotsLength()
+                    + fta.getFieldStartOffset(tupleIndex, fIdx);
+            tPointers[2 * f + 2] = fta.getFieldEndOffset(tupleIndex, fIdx) - fta.getFieldStartOffset(tupleIndex, fIdx);
+            if (f == 0) {
+                if (nmkComputer != null) {
+                    tPointers[0] = nmkComputer.normalize(b1, tPointers[1], tPointers[2]);
+                } else {
+                    tPointers[0] = 0;
+                }
+            }
+        }
     }
 }
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
index 7767ace..225f583 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
@@ -18,6 +18,7 @@
 import java.util.BitSet;
 import java.util.Comparator;
 
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
@@ -30,21 +31,25 @@
     private int nItems;
 
     private final Comparator<ReferenceEntry> comparator;
+    private final INormalizedKeyComputer nmkComputer;
+    private final int[] keyFields;
 
     public ReferencedPriorityQueue(int frameSize, RecordDescriptor recordDescriptor, int initSize,
-            Comparator<ReferenceEntry> comparator) {
+            Comparator<ReferenceEntry> comparator, int[] keyFields, INormalizedKeyComputer nmkComputer) {
         this.frameSize = frameSize;
         this.recordDescriptor = recordDescriptor;
         if (initSize < 1)
             throw new IllegalArgumentException();
         this.comparator = comparator;
+        this.nmkComputer = nmkComputer;
+        this.keyFields = keyFields;
         nItems = initSize;
         size = (initSize + 1) & 0xfffffffe;
         entries = new ReferenceEntry[size];
         runAvail = new BitSet(size);
         runAvail.set(0, initSize, true);
         for (int i = 0; i < size; i++) {
-            entries[i] = new ReferenceEntry(i, null, -1);
+            entries[i] = new ReferenceEntry(i, null, -1, keyFields, nmkComputer);
         }
     }
 
@@ -71,7 +76,7 @@
             entry.setAccessor(new FrameTupleAccessor(frameSize, recordDescriptor));
         }
         entry.getAccessor().reset(fta.getBuffer());
-        entry.setTupleIndex(tIndex);
+        entry.setTupleIndex(tIndex, keyFields, nmkComputer);
 
         add(entry);
         return entry.getRunid();
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
index 380ba29..a2ef99a 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
@@ -31,6 +31,7 @@
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -93,8 +94,8 @@
                         PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
                         new IBinaryComparatorFactory[] {
                                 PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0,
-                printer, 0);
+                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                        new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, printer, 0);
 
         runTest(spec);
     }
@@ -148,8 +149,8 @@
                         PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
                         new IBinaryComparatorFactory[] {
                                 PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0, filter,
-                0);
+                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                        new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, filter, 0);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), filter, 0, printer, 0);
 
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index 48db176..faa55e8 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -31,6 +31,7 @@
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -89,7 +90,8 @@
                         new int[] { 1 }, new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                 .of(UTF8StringPointable.FACTORY) }), new int[] { 1 },
                         new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                                .of(UTF8StringPointable.FACTORY) }), sorter, 0, printer, 0);
+                                .of(UTF8StringPointable.FACTORY) }, new UTF8StringNormalizedKeyComputerFactory()),
+                sorter, 0, printer, 0);
 
         runTest(spec);
     }
@@ -138,8 +140,8 @@
                         PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
                         new IBinaryComparatorFactory[] {
                                 PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }), sorter, 0,
-                printer, 0);
+                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                        new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, printer, 0);
 
         runTest(spec);
     }
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
index c61c15b..61c5a1c 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
@@ -156,8 +156,8 @@
         jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
         jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
                 new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
-                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
-                0, writeOperator, 0);
+                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, null),
+                sortOperator, 0, writeOperator, 0);
         jobSpec.addRoot(writeOperator);
 
         IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
index 3e43fb1..baa4dc7 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
@@ -163,8 +163,8 @@
         jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
         jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
                 new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
-                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
-                0, writeOperator, 0);
+                new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, null),
+                sortOperator, 0, writeOperator, 0);
         jobSpec.addRoot(writeOperator);
 
         IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
index c14f23a..3e4ecf8 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java
@@ -264,7 +264,7 @@
                 throw new TreeIndexException("Cannot bulk-load a non-empty tree.");
             }
 
-            this.cmp = MultiComparator.createIgnoreFieldLength(cmpFactories);
+            this.cmp = MultiComparator.create(cmpFactories);
 
             leafFrame.setMultiComparator(cmp);
             interiorFrame.setMultiComparator(cmp);
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
index 35c9bcb..8a3c313 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelper.java
@@ -23,7 +23,7 @@
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -35,17 +35,17 @@
     public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
         this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                opTrackerFactory, ioScheduler, ioOpCallbackFactory);
     }
 
     public LSMBTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
-            List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
-            ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
+            ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
         super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
-                ioScheduler, ioOpCallbackProvider);
+                ioScheduler, ioOpCallbackFactory);
     }
 
     @Override
@@ -55,6 +55,6 @@
                 opDesc.getStorageManager().getFileMapProvider(ctx), treeOpDesc.getTreeIndexTypeTraits(),
                 treeOpDesc.getTreeIndexComparatorFactories(), treeOpDesc.getTreeIndexBloomFilterKeyFields(),
                 bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory.getOperationTracker(ctx), ioScheduler,
-                ioOpCallbackProvider);
+                ioOpCallbackFactory.createIOOperationCallback());
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
index f4be88f..104a70d 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/dataflow/LSMBTreeDataflowHelperFactory.java
@@ -18,7 +18,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -31,10 +31,10 @@
 
     public LSMBTreeDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
             ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerFactory,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             double bloomFilterFalsePositiveRate) {
         super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerFactory, ioSchedulerProvider,
-                ioOpCallbackProvider, bloomFilterFalsePositiveRate);
+                ioOpCallbackFactory, bloomFilterFalsePositiveRate);
     }
 
     @Override
@@ -43,6 +43,6 @@
         return new LSMBTreeDataflowHelper(opDesc, ctx, partition,
                 virtualBufferCacheProvider.getVirtualBufferCaches(ctx), bloomFilterFalsePositiveRate,
                 mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
-                ioOpCallbackProvider);
+                ioOpCallbackFactory);
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index c8ed0d0..33ae7bb 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -92,9 +92,9 @@
             TreeIndexFactory<BTree> bulkLoadBTreeFactory, BloomFilterFactory bloomFilterFactory,
             double bloomFilterFalsePositiveRate, IFileMapProvider diskFileMapProvider, int fieldCount,
             IBinaryComparatorFactory[] cmpFactories, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback) {
         super(virtualBufferCaches, diskBTreeFactory.getBufferCache(), fileManager, diskFileMapProvider,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback);
         int i = 0;
         for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
             LSMBTreeMemoryComponent mutableComponent = new LSMBTreeMemoryComponent(new BTree(virtualBufferCache,
@@ -165,7 +165,7 @@
 
         if (flushOnExit) {
             BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
-                    ioOpCallbackProvider.getIOOperationCallback(this));
+                    ioOpCallback);
             ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             accessor.scheduleFlush(cb);
             try {
diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
index f43c8a6..8b363ff 100644
--- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeUtils.java
@@ -33,7 +33,7 @@
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeFileManager;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeCopyTupleWriterFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -49,7 +49,7 @@
             IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ITypeTraits[] typeTraits,
             IBinaryComparatorFactory[] cmpFactories, int[] bloomFilterKeyFields, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationCallback ioOpCallback) {
         LSMBTreeTupleWriterFactory insertTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
                 cmpFactories.length, false);
         LSMBTreeTupleWriterFactory deleteTupleWriterFactory = new LSMBTreeTupleWriterFactory(typeTraits,
@@ -78,7 +78,7 @@
         LSMBTree lsmTree = new LSMBTree(virtualBufferCaches, interiorFrameFactory, insertLeafFrameFactory,
                 deleteLeafFrameFactory, fileNameManager, diskBTreeFactory, bulkLoadBTreeFactory, bloomFilterFactory,
                 bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length, cmpFactories, mergePolicy,
-                opTracker, ioScheduler, ioOpCallbackProvider);
+                opTracker, ioScheduler, ioOpCallback);
         return lsmTree;
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
index 7086b59..9af08fa 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
@@ -17,12 +17,36 @@
 import java.util.List;
 
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
 
 public interface ILSMIOOperationCallback {
-    public void beforeOperation() throws HyracksDataException;
 
-    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+    /**
+     * This method is called on an IO operation sometime before the operation is executed.
+     * (i.e. IO operations could be flush or merge operations.)
+     */
+    public void beforeOperation(LSMOperationType opType) throws HyracksDataException;
+
+    /**
+     * This method is called on an IO operation sometime after the operation was completed.
+     * (i.e. IO operations could be flush or merge operations.)
+     * 
+     * @param oldComponents
+     * @param newComponent
+     * @throws HyracksDataException
+     */
+    public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
             throws HyracksDataException;
 
-    public void afterFinalize(ILSMComponent newComponent) throws HyracksDataException;
+    /**
+     * This method is called on an IO operation when the operation needs any cleanup works
+     * regardless that the IO operation was executed or not. Once the IO operation is executed,
+     * this method should be called after ILSMIOOperationCallback.afterOperation() was called.
+     * 
+     * @param newComponent
+     * @throws HyracksDataException
+     */
+    public void afterFinalize(LSMOperationType opType, ILSMComponent newComponent) throws HyracksDataException;
+
+    public void setNumOfMutableComponents(int count);
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
index 6dbce32..804142e 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
@@ -17,5 +17,5 @@
 import java.io.Serializable;
 
 public interface ILSMIOOperationCallbackFactory extends Serializable {
-    public ILSMIOOperationCallback createIOOperationCallback(Object syncObj);
+    public ILSMIOOperationCallback createIOOperationCallback();
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
index 416a15b..d37a105 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/api/ILSMIndex.java
@@ -41,6 +41,8 @@
     public ILSMOperationTracker getOperationTracker();
 
     public ILSMIOOperationScheduler getIOScheduler();
+    
+    public ILSMIOOperationCallback getIOOperationCallback();
 
     public List<ILSMComponent> getImmutableComponents();
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
index b1bf6b4..e277edc 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelper.java
@@ -20,7 +20,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -36,26 +36,26 @@
     protected final ILSMMergePolicy mergePolicy;
     protected final ILSMIOOperationScheduler ioScheduler;
     protected final ILSMOperationTrackerProvider opTrackerFactory;
-    protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+    protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
 
     public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
         this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                opTrackerFactory, ioScheduler, ioOpCallbackFactory);
     }
 
     public AbstractLSMIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
         super(opDesc, ctx, partition);
         this.virtualBufferCaches = virtualBufferCaches;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.mergePolicy = mergePolicy;
         this.opTrackerFactory = opTrackerFactory;
         this.ioScheduler = ioScheduler;
-        this.ioOpCallbackProvider = ioOpCallbackProvider;
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
index 205bf27..2c082bb 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/AbstractLSMIndexDataflowHelperFactory.java
@@ -16,7 +16,7 @@
 package edu.uci.ics.hyracks.storage.am.lsm.common.dataflow;
 
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -29,18 +29,18 @@
     protected final ILSMMergePolicyProvider mergePolicyProvider;
     protected final ILSMOperationTrackerProvider opTrackerFactory;
     protected final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
-    protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+    protected final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
     protected final double bloomFilterFalsePositiveRate;
 
     public AbstractLSMIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
             ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerFactory,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             double bloomFilterFalsePositiveRate) {
         this.virtualBufferCacheProvider = virtualBufferCacheProvider;
         this.mergePolicyProvider = mergePolicyProvider;
         this.opTrackerFactory = opTrackerFactory;
         this.ioSchedulerProvider = ioSchedulerProvider;
-        this.ioOpCallbackProvider = ioOpCallbackProvider;
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index 794f440..44bcfc2 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -26,6 +26,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
@@ -42,7 +43,7 @@
     protected final ILSMHarness lsmHarness;
 
     protected final ILSMIOOperationScheduler ioScheduler;
-    protected final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+    protected final ILSMIOOperationCallback ioOpCallback;
 
     // In-memory components.   
     protected final List<ILSMComponent> memoryComponents;
@@ -63,14 +64,15 @@
     public AbstractLSMIndex(List<IVirtualBufferCache> virtualBufferCaches, IBufferCache diskBufferCache,
             ILSMIndexFileManager fileManager, IFileMapProvider diskFileMapProvider,
             double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback) {
         this.virtualBufferCaches = virtualBufferCaches;
         this.diskBufferCache = diskBufferCache;
         this.diskFileMapProvider = diskFileMapProvider;
         this.fileManager = fileManager;
         this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
         this.ioScheduler = ioScheduler;
-        this.ioOpCallbackProvider = ioOpCallbackProvider;
+        this.ioOpCallback = ioOpCallback;
+        this.ioOpCallback.setNumOfMutableComponents(virtualBufferCaches.size());
         lsmHarness = new LSMHarness(this, mergePolicy, opTracker);
         isActivated = false;
         diskComponents = new LinkedList<ILSMComponent>();
@@ -179,6 +181,11 @@
     }
 
     @Override
+    public ILSMIOOperationCallback getIOOperationCallback() {
+        return ioOpCallback;
+    }
+
+    @Override
     public IBufferCache getBufferCache() {
         return diskBufferCache;
     }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
index a7ca95f..fefe812 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
@@ -38,20 +38,25 @@
     }
 
     @Override
-    public void beforeOperation() throws HyracksDataException {
-        wrappedCallback.beforeOperation();
+    public void beforeOperation(LSMOperationType opType) throws HyracksDataException {
+        wrappedCallback.beforeOperation(opType);
     }
 
     @Override
-    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+    public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
             throws HyracksDataException {
-        wrappedCallback.afterOperation(oldComponents, newComponent);
+        wrappedCallback.afterOperation(opType, oldComponents, newComponent);
     }
 
     @Override
-    public synchronized void afterFinalize(ILSMComponent newComponent) throws HyracksDataException {
-        wrappedCallback.afterFinalize(newComponent);
+    public synchronized void afterFinalize(LSMOperationType opType, ILSMComponent newComponent) throws HyracksDataException {
+        wrappedCallback.afterFinalize(opType, newComponent);
         notifyAll();
         notified = true;
     }
+
+    @Override
+    public void setNumOfMutableComponents(int count) {
+        wrappedCallback.setNumOfMutableComponents(count);
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 145bfe9..ca775b7 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -118,6 +118,7 @@
         // Check if there is any action that is needed to be taken based on the operation type
         switch (opType) {
             case FLUSH:
+                lsmIndex.getIOOperationCallback().beforeOperation(LSMOperationType.FLUSH);
                 // Changing the flush status should *always* precede changing the mutable component.
                 lsmIndex.changeFlushStatusForCurrentMutableCompoent(false);
                 lsmIndex.changeMutableComponent();
@@ -125,6 +126,8 @@
                 // again if they can grab and enter the mutable component.
                 opTracker.notifyAll();
                 break;
+            case MERGE:
+                lsmIndex.getIOOperationCallback().beforeOperation(LSMOperationType.MERGE);
             default:
                 break;
         }
@@ -254,9 +257,7 @@
     public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
             throws HyracksDataException {
         if (!getAndEnterComponents(ctx, LSMOperationType.FLUSH, true)) {
-            callback.beforeOperation();
-            callback.afterOperation(null, null);
-            callback.afterFinalize(null);
+            callback.afterFinalize(LSMOperationType.FLUSH, null);
             return;
         }
         lsmIndex.scheduleFlush(ctx, callback);
@@ -271,13 +272,12 @@
 
         ILSMComponent newComponent = null;
         try {
-            operation.getCallback().beforeOperation();
             newComponent = lsmIndex.flush(operation);
-            operation.getCallback().afterOperation(null, newComponent);
+            operation.getCallback().afterOperation(LSMOperationType.FLUSH, null, newComponent);
             lsmIndex.markAsValid(newComponent);
         } finally {
             exitComponents(ctx, LSMOperationType.FLUSH, newComponent, false);
-            operation.getCallback().afterFinalize(newComponent);
+            operation.getCallback().afterFinalize(LSMOperationType.FLUSH, newComponent);
         }
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Finished the flush operation for index: " + lsmIndex);
@@ -289,9 +289,7 @@
             throws HyracksDataException, IndexException {
         // Merge should always be a try operation, because it should never fail to enter the components unless the merge policy is erroneous.
         if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
-            callback.beforeOperation();
-            callback.afterOperation(null, null);
-            callback.afterFinalize(null);
+            callback.afterFinalize(LSMOperationType.MERGE, null);
             return;
         }
         lsmIndex.scheduleMerge(ctx, callback);
@@ -306,13 +304,12 @@
 
         ILSMComponent newComponent = null;
         try {
-            operation.getCallback().beforeOperation();
             newComponent = lsmIndex.merge(operation);
-            operation.getCallback().afterOperation(ctx.getComponentHolder(), newComponent);
+            operation.getCallback().afterOperation(LSMOperationType.MERGE, ctx.getComponentHolder(), newComponent);
             lsmIndex.markAsValid(newComponent);
         } finally {
             exitComponents(ctx, LSMOperationType.MERGE, newComponent, false);
-            operation.getCallback().afterFinalize(newComponent);
+            operation.getCallback().afterFinalize(LSMOperationType.MERGE, newComponent);
         }
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Finished the merge operation for index: " + lsmIndex);
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
index a43b44b..2c8369e 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallback.java
@@ -19,25 +19,26 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 
-public enum NoOpIOOperationCallback implements ILSMIOOperationCallback, ILSMIOOperationCallbackProvider {
+public enum NoOpIOOperationCallback implements ILSMIOOperationCallback, ILSMIOOperationCallbackProvider, ILSMIOOperationCallbackFactory {
     INSTANCE;
 
     @Override
-    public void beforeOperation() throws HyracksDataException {
+    public void beforeOperation(LSMOperationType opType) throws HyracksDataException {
         // Do nothing.
     }
 
     @Override
-    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+    public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
             throws HyracksDataException {
         // Do nothing.
     }
 
     @Override
-    public void afterFinalize(ILSMComponent newComponent) throws HyracksDataException {
+    public void afterFinalize(LSMOperationType opType, ILSMComponent newComponent) throws HyracksDataException {
         // Do nothing.
     }
 
@@ -45,4 +46,14 @@
     public ILSMIOOperationCallback getIOOperationCallback(ILSMIndex index) {
         return INSTANCE;
     }
+
+    @Override
+    public ILSMIOOperationCallback createIOOperationCallback() {
+        return INSTANCE;
+    }
+
+    @Override
+    public void setNumOfMutableComponents(int count) {
+        // Do nothing.
+    }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
index ee8febe..ed2f8cf 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelper.java
@@ -21,7 +21,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -38,17 +38,17 @@
     public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
         this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                opTrackerFactory, ioScheduler, ioOpCallbackFactory);
     }
 
     public LSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
             List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
         super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
-                ioScheduler, ioOpCallbackProvider);
+                ioScheduler, ioOpCallbackFactory);
     }
 
     @Override
@@ -62,7 +62,8 @@
                     invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
                     invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
                     diskBufferCache, file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
-                    opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider);
+                    opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+                    ioOpCallbackFactory.createIOOperationCallback());
             return invIndex;
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
index 85aa312..84c7150 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexDataflowHelperFactory.java
@@ -18,7 +18,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -31,10 +31,10 @@
 
     public LSMInvertedIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
             ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerProvider,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             double bloomFilterFalsePositiveRate) {
         super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerProvider, ioSchedulerProvider,
-                ioOpCallbackProvider, bloomFilterFalsePositiveRate);
+                ioOpCallbackFactory, bloomFilterFalsePositiveRate);
     }
 
     @Override
@@ -43,7 +43,7 @@
         return new LSMInvertedIndexDataflowHelper(opDesc, ctx, partition,
                 virtualBufferCacheProvider.getVirtualBufferCaches(ctx), bloomFilterFalsePositiveRate,
                 mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
-                ioOpCallbackProvider);
+                ioOpCallbackFactory);
     }
 
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
index 1acb437..57a41ca 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelper.java
@@ -21,7 +21,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -38,17 +38,17 @@
     public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, List<IVirtualBufferCache> virtualBufferCache, ILSMMergePolicy mergePolicy,
             ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
         this(opDesc, ctx, partition, virtualBufferCache, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE, mergePolicy,
-                opTrackerFactory, ioScheduler, ioOpCallbackProvider);
+                opTrackerFactory, ioScheduler, ioOpCallbackFactory);
     }
 
     public PartitionedLSMInvertedIndexDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, List<IVirtualBufferCache> virtualBufferCaches, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
         super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
-                ioScheduler, ioOpCallbackProvider);
+                ioScheduler, ioOpCallbackFactory);
     }
 
     @Override
@@ -62,7 +62,8 @@
                     invIndexOpDesc.getInvListsComparatorFactories(), invIndexOpDesc.getTokenTypeTraits(),
                     invIndexOpDesc.getTokenComparatorFactories(), invIndexOpDesc.getTokenizerFactory(),
                     diskBufferCache, file.getFile().getPath(), bloomFilterFalsePositiveRate, mergePolicy,
-                    opTrackerFactory.getOperationTracker(ctx), ioScheduler, ioOpCallbackProvider);
+                    opTrackerFactory.getOperationTracker(ctx), ioScheduler,
+                    ioOpCallbackFactory.createIOOperationCallback());
             return invIndex;
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
index 63d99aa..d78ae7e 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/PartitionedLSMInvertedIndexDataflowHelperFactory.java
@@ -18,7 +18,7 @@
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -31,10 +31,10 @@
 
     public PartitionedLSMInvertedIndexDataflowHelperFactory(IVirtualBufferCacheProvider virtualBufferCacheProvider,
             ILSMMergePolicyProvider mergePolicyProvider, ILSMOperationTrackerProvider opTrackerProvider,
-            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             double bloomFilterFalsePositiveRate) {
         super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerProvider, ioSchedulerProvider,
-                ioOpCallbackProvider, bloomFilterFalsePositiveRate);
+                ioOpCallbackFactory, bloomFilterFalsePositiveRate);
     }
 
     @Override
@@ -43,6 +43,6 @@
         return new PartitionedLSMInvertedIndexDataflowHelper(opDesc, ctx, partition,
                 virtualBufferCacheProvider.getVirtualBufferCaches(ctx), bloomFilterFalsePositiveRate,
                 mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
-                ioOpCallbackProvider);
+                ioOpCallbackFactory);
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index f2f058a..cef6aee 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -52,7 +52,6 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessorInternal;
@@ -100,9 +99,9 @@
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+            ILSMIOOperationCallback ioOpCallback) throws IndexException {
         super(virtualBufferCaches, diskInvIndexFactory.getBufferCache(), fileManager, diskFileMapProvider,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback);
 
         this.tokenizerFactory = tokenizerFactory;
         this.invListTypeTraits = invListTypeTraits;
@@ -211,7 +210,7 @@
         isActivated = false;
         if (flushOnExit) {
             BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
-                    ioOpCallbackProvider.getIOOperationCallback(this));
+                    ioOpCallback);
             ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             accessor.scheduleFlush(cb);
             try {
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index 5b0c48d..ddb6060 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -24,7 +24,7 @@
 import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.BloomFilterFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.IVirtualFreePageManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -46,11 +46,10 @@
             IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+            ILSMIOOperationCallback ioOpCallback) throws IndexException {
         super(virtualBufferCaches, diskInvIndexFactory, deletedKeysBTreeFactory, bloomFilterFactory,
                 bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits, invListCmpFactories,
-                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler,
-                ioOpCallbackProvider);
+                tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallback);
     }
 
     @Override
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index ed02ba1..372221e 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -33,7 +33,7 @@
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManagerFactory;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -122,7 +122,7 @@
             IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
             IBufferCache diskBufferCache, String onDiskDir, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws IndexException {
+            ILSMIOOperationCallback ioOpCallback) throws IndexException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
                 invListCmpFactories, diskBufferCache);
@@ -147,7 +147,7 @@
         LSMInvertedIndex invIndex = new LSMInvertedIndex(virtualBufferCaches, invIndexFactory, deletedKeysBTreeFactory,
                 bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager, diskFileMapProvider, invListTypeTraits,
                 invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
-                ioScheduler, ioOpCallbackProvider);
+                ioScheduler, ioOpCallback);
         return invIndex;
     }
 
@@ -157,8 +157,7 @@
             ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
             IBinaryTokenizerFactory tokenizerFactory, IBufferCache diskBufferCache, String onDiskDir,
             double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
-            throws IndexException {
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback) throws IndexException {
 
         BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(diskFileMapProvider, invListTypeTraits,
                 invListCmpFactories, diskBufferCache);
@@ -183,7 +182,7 @@
         PartitionedLSMInvertedIndex invIndex = new PartitionedLSMInvertedIndex(virtualBufferCaches, invIndexFactory,
                 deletedKeysBTreeFactory, bloomFilterFactory, bloomFilterFalsePositiveRate, fileManager,
                 diskFileMapProvider, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
-                tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+                tokenizerFactory, mergePolicy, opTracker, ioScheduler, ioOpCallback);
         return invIndex;
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
index 5d57aae..636f0cb 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/AbstractLSMRTreeDataflowHelper.java
@@ -27,7 +27,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -49,11 +49,11 @@
             List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory) {
         this(opDesc, ctx, partition, virtualBufferCaches, DEFAULT_BLOOM_FILTER_FALSE_POSITIVE_RATE,
                 btreeComparatorFactories, valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory,
-                ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+                ioScheduler, ioOpCallbackFactory, linearizeCmpFactory);
     }
 
     public AbstractLSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
@@ -61,10 +61,10 @@
             IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory) {
         super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, mergePolicy, opTrackerFactory,
-                ioScheduler, ioOpCallbackProvider);
+                ioScheduler, ioOpCallbackFactory);
         this.btreeComparatorFactories = btreeComparatorFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
index bc4bca3..92364fa 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelper.java
@@ -27,7 +27,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -44,10 +44,10 @@
             List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory) {
         super(opDesc, ctx, partition, virtualBufferCaches, btreeComparatorFactories, valueProviderFactories,
-                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackFactory, linearizeCmpFactory);
     }
 
     public LSMRTreeDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition,
@@ -55,11 +55,11 @@
             IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory) {
         super(opDesc, ctx, partition, virtualBufferCaches, bloomFilterFalsePositiveRate, btreeComparatorFactories,
                 valueProviderFactories, rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler,
-                ioOpCallbackProvider, linearizeCmpFactory);
+                ioOpCallbackFactory, linearizeCmpFactory);
     }
 
     @Override
@@ -72,8 +72,8 @@
         try {
             return LSMRTreeUtils.createLSMTree(virtualBufferCaches, file, diskBufferCache, diskFileMapProvider,
                     typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                    bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
-                    linearizeCmpFactory);
+                    bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler,
+                    ioOpCallbackFactory.createIOOperationCallback(), linearizeCmpFactory);
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
index 99b4219..8254689 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeDataflowHelperFactory.java
@@ -21,7 +21,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -42,10 +42,10 @@
             RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
             IVirtualBufferCacheProvider virtualBufferCacheProvider, ILSMMergePolicyProvider mergePolicyProvider,
             ILSMOperationTrackerProvider opTrackerFactory, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory,
             double bloomFilterFalsePositiveRate) {
         super(virtualBufferCacheProvider, mergePolicyProvider, opTrackerFactory, ioSchedulerProvider,
-                ioOpCallbackProvider, bloomFilterFalsePositiveRate);
+                ioOpCallbackFactory, bloomFilterFalsePositiveRate);
         this.btreeComparatorFactories = btreeComparatorFactories;
         this.valueProviderFactories = valueProviderFactories;
         this.rtreePolicyType = rtreePolicyType;
@@ -59,6 +59,6 @@
                 virtualBufferCacheProvider.getVirtualBufferCaches(ctx), bloomFilterFalsePositiveRate,
                 btreeComparatorFactories, valueProviderFactories, rtreePolicyType,
                 mergePolicyProvider.getMergePolicy(ctx), opTrackerFactory, ioSchedulerProvider.getIOScheduler(ctx),
-                ioOpCallbackProvider, linearizeCmpFactory);
+                ioOpCallbackFactory, linearizeCmpFactory);
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
index 72f3480..f430456 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelper.java
@@ -27,7 +27,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -40,13 +40,14 @@
 
 public class LSMRTreeWithAntiMatterTuplesDataflowHelper extends AbstractLSMRTreeDataflowHelper {
     public LSMRTreeWithAntiMatterTuplesDataflowHelper(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
-            int partition, List<IVirtualBufferCache> virtualBufferCaches, IBinaryComparatorFactory[] btreeComparatorFactories,
+            int partition, List<IVirtualBufferCache> virtualBufferCaches,
+            IBinaryComparatorFactory[] btreeComparatorFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTrackerProvider opTrackerFactory,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
             ILinearizeComparatorFactory linearizeCmpFactory) {
         super(opDesc, ctx, partition, virtualBufferCaches, btreeComparatorFactories, valueProviderFactories,
-                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+                rtreePolicyType, mergePolicy, opTrackerFactory, ioScheduler, ioOpCallbackFactory, linearizeCmpFactory);
     }
 
     @Override
@@ -59,7 +60,8 @@
         try {
             return LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCaches, file, diskBufferCache,
                     diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories,
-                    rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider, linearizeCmpFactory);
+                    rtreePolicyType, mergePolicy, opTracker, ioScheduler,
+                    ioOpCallbackFactory.createIOOperationCallback(), linearizeCmpFactory);
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
index 00268b3..ef34876 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/dataflow/LSMRTreeWithAntiMatterTuplesDataflowHelperFactory.java
@@ -22,7 +22,7 @@
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
@@ -40,14 +40,14 @@
     private final ILSMMergePolicyProvider mergePolicyProvider;
     private final ILSMOperationTrackerProvider opTrackerProvider;
     private final ILSMIOOperationSchedulerProvider ioSchedulerProvider;
-    private final ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+    private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
     private final ILinearizeComparatorFactory linearizeCmpFactory;
 
     public LSMRTreeWithAntiMatterTuplesDataflowHelperFactory(IPrimitiveValueProviderFactory[] valueProviderFactories,
             RTreePolicyType rtreePolicyType, IBinaryComparatorFactory[] btreeComparatorFactories,
             IVirtualBufferCacheProvider virtualBufferCacheProvider, ILSMMergePolicyProvider mergePolicyProvider,
             ILSMOperationTrackerProvider opTrackerProvider, ILSMIOOperationSchedulerProvider ioSchedulerProvider,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizeCmpFactory) {
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILinearizeComparatorFactory linearizeCmpFactory) {
         this.virtualBufferCacheProvider = virtualBufferCacheProvider;
         this.btreeComparatorFactories = btreeComparatorFactories;
         this.valueProviderFactories = valueProviderFactories;
@@ -55,7 +55,7 @@
         this.mergePolicyProvider = mergePolicyProvider;
         this.ioSchedulerProvider = ioSchedulerProvider;
         this.opTrackerProvider = opTrackerProvider;
-        this.ioOpCallbackProvider = ioOpCallbackProvider;
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
         this.linearizeCmpFactory = linearizeCmpFactory;
     }
 
@@ -65,6 +65,6 @@
         return new LSMRTreeWithAntiMatterTuplesDataflowHelper(opDesc, ctx, partition,
                 virtualBufferCacheProvider.getVirtualBufferCaches(ctx), btreeComparatorFactories,
                 valueProviderFactories, rtreePolicyType, mergePolicyProvider.getMergePolicy(ctx), opTrackerProvider,
-                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackProvider, linearizeCmpFactory);
+                ioSchedulerProvider.getIOScheduler(ctx), ioOpCallbackFactory, linearizeCmpFactory);
     }
 }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index 04118c8..b09e115 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -40,7 +40,7 @@
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponentFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
@@ -87,9 +87,9 @@
             IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
             int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, double bloomFilterFalsePositiveRate,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationCallback ioOpCallback) {
         super(virtualBufferCaches, componentFactory.getBufferCache(), fileManager, diskFileMapProvider,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback);
         int i = 0;
         for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
             RTree memRTree = new RTree(virtualBufferCache,
@@ -156,7 +156,7 @@
 
         if (flushOnExit) {
             BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
-                    ioOpCallbackProvider.getIOOperationCallback(this));
+                    ioOpCallback);
             ILSMIndexAccessor accessor = createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
             accessor.scheduleFlush(cb);
             try {
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 410fde8..a4b6139 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -74,12 +74,12 @@
             IBinaryComparatorFactory[] btreeCmpFactories, ILinearizeComparatorFactory linearizer,
             int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray, ILSMMergePolicy mergePolicy,
             ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationCallback ioOpCallback) {
         super(virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
                 btreeLeafFrameFactory, fileNameManager, new LSMRTreeDiskComponentFactory(diskRTreeFactory,
                         diskBTreeFactory, bloomFilterFactory), diskFileMapProvider, fieldCount, rtreeCmpFactories,
                 btreeCmpFactories, linearizer, comparatorFields, linearizerArray, bloomFilterFalsePositiveRate,
-                mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+                mergePolicy, opTracker, ioScheduler, ioOpCallback);
     }
 
     /**
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index 0996257..812e942 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -74,11 +74,11 @@
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
             ILinearizeComparatorFactory linearizer, int[] comparatorFields, IBinaryComparatorFactory[] linearizerArray,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) {
+            ILSMIOOperationCallback ioOpCallback) {
         super(virtualBufferCaches, rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory,
                 btreeLeafFrameFactory, fileManager, new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(diskRTreeFactory),
                 diskFileMapProvider, fieldCount, rtreeCmpFactories, btreeCmpFactories, linearizer, comparatorFields,
-                linearizerArray, 0, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+                linearizerArray, 0, mergePolicy, opTracker, ioScheduler, ioOpCallback);
         bulkLoaComponentFactory = new LSMRTreeWithAntiMatterTuplesDiskComponentFactory(bulkLoadRTreeFactory);
         this.bTreeTupleSorter = null;
     }
diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
index b16d2b6..ee942e7 100644
--- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
+++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/utils/LSMRTreeUtils.java
@@ -33,7 +33,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManagerFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -65,7 +65,7 @@
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider,
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback,
             ILinearizeComparatorFactory linearizeCmpFactory) throws TreeIndexException {
         LSMTypeAwareTupleWriterFactory rtreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, false);
         LSMTypeAwareTupleWriterFactory btreeTupleWriterFactory = new LSMTypeAwareTupleWriterFactory(typeTraits, true);
@@ -105,7 +105,7 @@
                 btreeInteriorFrameFactory, btreeLeafFrameFactory, fileNameManager, diskRTreeFactory, diskBTreeFactory,
                 bloomFilterFactory, bloomFilterFalsePositiveRate, diskFileMapProvider, typeTraits.length,
                 rtreeCmpFactories, btreeCmpFactories, linearizeCmpFactory, comparatorFields, linearizerArray,
-                mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+                mergePolicy, opTracker, ioScheduler, ioOpCallback);
         return lsmTree;
     }
 
@@ -115,7 +115,7 @@
             IBinaryComparatorFactory[] rtreeCmpFactories, IBinaryComparatorFactory[] btreeCmpFactories,
             IPrimitiveValueProviderFactory[] valueProviderFactories, RTreePolicyType rtreePolicyType,
             ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider, ILinearizeComparatorFactory linearizerCmpFactory)
+            ILSMIOOperationCallback ioOpCallback, ILinearizeComparatorFactory linearizerCmpFactory)
             throws TreeIndexException {
         LSMRTreeTupleWriterFactory rtreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, false);
         LSMRTreeTupleWriterFactory btreeTupleWriterFactory = new LSMRTreeTupleWriterFactory(typeTraits, true);
@@ -157,7 +157,7 @@
                 rtreeInteriorFrameFactory, rtreeLeafFrameFactory, btreeInteriorFrameFactory, btreeLeafFrameFactory,
                 fileNameManager, diskRTreeFactory, bulkLoadRTreeFactory, diskFileMapProvider, typeTraits.length,
                 rtreeCmpFactories, btreeCmpFactories, linearizerCmpFactory, comparatorFields, linearizerArray,
-                mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider);
+                mergePolicy, opTracker, ioScheduler, ioOpCallback);
         return lsmTree;
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
index 753be5d..447c507 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeBulkLoadTest.java
@@ -54,7 +54,7 @@
         return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
index 646343a..0303dbd 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeDeleteTest.java
@@ -54,7 +54,7 @@
         return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
index b0617a1..dc55f5d 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeExamplesTest.java
@@ -37,7 +37,7 @@
         return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
                 bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Before
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
index bce69a2..4fc3960 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeInsertTest.java
@@ -54,7 +54,7 @@
         return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
index 0f903e5..f553b72 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeLifecycleTest.java
@@ -56,7 +56,7 @@
         testCtx = LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, fieldSerdes.length,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallback());
         index = testCtx.getIndex();
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
index ab46f3f..f5cfb32 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMergeTest.java
@@ -53,7 +53,7 @@
         return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index cc3389f..2234075 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -48,7 +48,7 @@
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
index d7d29a8..277df6b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeMultiBulkLoadTest.java
@@ -55,7 +55,7 @@
         return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
index ce9ae0e..f1a5355 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java
@@ -55,7 +55,7 @@
                 SerdeUtils.serdesToComparatorFactories(keySerdes, keySerdes.length), bloomFilterKeyFields,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
                 NoOpOperationTrackerProvider.INSTANCE.getOperationTracker(null), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider());
+                harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
index 6f8ef84..a5097b4 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeUpdateTest.java
@@ -54,7 +54,7 @@
         return LSMBTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, numKeys,
                 harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
-                harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
index 1ae201f..adee618 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeMultiThreadTest.java
@@ -53,7 +53,7 @@
         return LSMBTreeUtils.createLSMTree(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, cmpFactories,
                 bloomFilterKeyFields, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
index 39e7184..6f7c23a 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestContext.java
@@ -28,7 +28,7 @@
 import edu.uci.ics.hyracks.storage.am.common.CheckTuple;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -66,8 +66,8 @@
     public static LSMBTreeTestContext create(List<IVirtualBufferCache> virtualBufferCaches, FileReference file,
             IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
             int numKeyFields, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
-            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws Exception {
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback)
+            throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] cmpFactories = SerdeUtils.serdesToComparatorFactories(fieldSerdes, numKeyFields);
         int[] bloomFilterKeyFields = new int[numKeyFields];
@@ -76,7 +76,7 @@
         }
         LSMBTree lsmTree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, diskBufferCache, diskFileMapProvider,
                 typeTraits, cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, mergePolicy, opTracker,
-                ioScheduler, ioOpCallbackProvider);
+                ioScheduler, ioOpCallback);
         LSMBTreeTestContext testCtx = new LSMBTreeTestContext(fieldSerdes, lsmTree);
         return testCtx;
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
index 216861f..b88da7b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -32,7 +32,7 @@
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType;
 import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -73,7 +73,7 @@
     protected ILSMIOOperationScheduler ioScheduler;
     protected ILSMMergePolicy mergePolicy;
     protected ILSMOperationTracker opTracker;
-    protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+    protected ILSMIOOperationCallback ioOpCallback;
 
     protected final Random rnd = new Random();
     protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
@@ -92,7 +92,7 @@
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTracker = new ThreadCountingTracker();
-        this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
+        this.ioOpCallback = NoOpIOOperationCallback.INSTANCE;
         this.numMutableComponents = AccessMethodTestsConfig.LSM_BTREE_NUM_MUTABLE_COMPONENTS;
     }
 
@@ -206,7 +206,7 @@
         return mergePolicy;
     }
 
-    public ILSMIOOperationCallbackProvider getIOOperationCallbackProvider() {
-        return ioOpCallbackProvider;
+    public ILSMIOOperationCallback getIOOperationCallback() {
+        return ioOpCallback;
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index 48176b7..0907fc0 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -30,7 +30,7 @@
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -69,7 +69,7 @@
     protected ILSMIOOperationScheduler ioScheduler;
     protected ILSMMergePolicy mergePolicy;
     protected ILSMOperationTracker opTracker;
-    protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+    protected ILSMIOOperationCallback ioOpCallback;
 
     protected final Random rnd = new Random();
     protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
@@ -90,7 +90,7 @@
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTracker = new ThreadCountingTracker();
-        this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
+        this.ioOpCallback = NoOpIOOperationCallback.INSTANCE;
         this.numMutableComponents = AccessMethodTestsConfig.LSM_INVINDEX_NUM_MUTABLE_COMPONENTS;
     }
 
@@ -212,7 +212,7 @@
         return mergePolicy;
     }
 
-    public ILSMIOOperationCallbackProvider getIOOperationCallbackProvider() {
-        return ioOpCallbackProvider;
+    public ILSMIOOperationCallback getIOOperationCallback() {
+        return ioOpCallback;
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 5edf74e..059df73 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -152,8 +152,7 @@
                         harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
                         tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
                         harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                        harness.getOperationTracker(), harness.getIOScheduler(),
-                        harness.getIOOperationCallbackProvider());
+                        harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
                 break;
             }
             case PARTITIONED_LSM: {
@@ -161,8 +160,7 @@
                         harness.getDiskFileMapProvider(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
                         tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
                         harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                        harness.getOperationTracker(), harness.getIOScheduler(),
-                        harness.getIOOperationCallbackProvider());
+                        harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
                 break;
             }
             default: {
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
index 09daaab..dadd08c 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeBulkLoadTest.java
@@ -57,7 +57,7 @@
         return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
index f5b4ae4..0b92cd3 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeDeleteTest.java
@@ -57,7 +57,7 @@
         return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
index c94bb48..b04ae16 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeExamplesTest.java
@@ -41,7 +41,7 @@
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
                 btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
                 harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(),
+                harness.getIOOperationCallback(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
index 858f90a..35c6a6b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeInsertTest.java
@@ -57,7 +57,7 @@
         return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
index 61692b8..6ef2e94 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeLifecycleTest.java
@@ -66,7 +66,7 @@
         testCtx = LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, RTreePolicyType.RTREE, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
         index = testCtx.getIndex();
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
index 436f9b1..94528e7 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTest.java
@@ -56,7 +56,7 @@
         return LSMRTreeTestContext.create(harness.getVirtualBufferCaches(), harness.getFileReference(),
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), fieldSerdes, valueProviderFactories,
                 numKeys, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
index 97a3dcc..22a6eb8 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesBulkLoadTest.java
@@ -57,7 +57,7 @@
         return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
index 843dcc7..10aec7e 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -57,7 +57,7 @@
         return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
index 6ffa59a..a51abef 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesExamplesTest.java
@@ -41,7 +41,7 @@
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
                 rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
                 harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(),
+                harness.getIOOperationCallback(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
index 1e1f46d..e5b33cc 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesInsertTest.java
@@ -57,7 +57,7 @@
         return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
index 607e980..10ae5e1 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesLifecycleTest.java
@@ -66,7 +66,7 @@
         testCtx = LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 fieldSerdes, valueProviderFactories, numKeys, RTreePolicyType.RTREE, harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
         index = testCtx.getIndex();
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
index bdd80aa..21d3759 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesMergeTest.java
@@ -56,7 +56,7 @@
         return LSMRTreeWithAntiMatterTuplesTestContext.create(harness.getVirtualBufferCaches(),
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(),
                 fieldSerdes, valueProviderFactories, numKeys, rtreePolicyType, harness.getMergePolicy(),
-                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallbackProvider());
+                harness.getOperationTracker(), harness.getIOScheduler(), harness.getIOOperationCallback());
     }
 
     @Override
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
index 0c0d804..077bd45 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeMultiThreadTest.java
@@ -61,7 +61,7 @@
                 harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits, rtreeCmpFactories,
                 btreeCmpFactories, valueProviderFactories, rtreePolicyType, harness.getBoomFilterFalsePositiveRate(),
                 harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(),
+                harness.getIOOperationCallback(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
     }
 
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
index 8b54373..ef871dd 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesMultiThreadTest.java
@@ -61,7 +61,7 @@
                 harness.getFileReference(), harness.getDiskBufferCache(), harness.getDiskFileMapProvider(), typeTraits,
                 rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
                 harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
-                harness.getIOOperationCallbackProvider(),
+                harness.getIOOperationCallback(),
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
 
     }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
index 9536e96..42e793b 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestContext.java
@@ -26,7 +26,7 @@
 import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -71,8 +71,7 @@
             IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider, ISerializerDeserializer[] fieldSerdes,
             IPrimitiveValueProviderFactory[] valueProviderFactories, int numKeyFields, RTreePolicyType rtreePolicyType,
             double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
-            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackProvider ioOpCallbackProvider)
-            throws Exception {
+            ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback) throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
                 .serdesToComparatorFactories(fieldSerdes, numKeyFields);
@@ -80,7 +79,7 @@
                 fieldSerdes.length);
         LSMRTree lsmTree = LSMRTreeUtils.createLSMTree(virtualBufferCaches, file, diskBufferCache, diskFileMapProvider,
                 typeTraits, rtreeCmpFactories, btreeCmpFactories, valueProviderFactories, rtreePolicyType,
-                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+                bloomFilterFalsePositiveRate, mergePolicy, opTracker, ioScheduler, ioOpCallback,
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
         LSMRTreeTestContext testCtx = new LSMRTreeTestContext(fieldSerdes, lsmTree);
         return testCtx;
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
index b67274e..66e79a2 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -31,7 +31,7 @@
 import edu.uci.ics.hyracks.api.io.IODeviceHandle;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.storage.am.config.AccessMethodTestsConfig;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -68,7 +68,7 @@
     protected List<IVirtualBufferCache> virtualBufferCaches;
     protected IHyracksTaskContext ctx;
     protected ILSMIOOperationScheduler ioScheduler;
-    protected ILSMIOOperationCallbackProvider ioOpCallbackProvider;
+    protected ILSMIOOperationCallback ioOpCallback;
     protected ILSMMergePolicy mergePolicy;
     protected ILSMOperationTracker opTracker;
 
@@ -89,7 +89,7 @@
         this.ioScheduler = SynchronousScheduler.INSTANCE;
         this.mergePolicy = NoMergePolicy.INSTANCE;
         this.opTracker = new ThreadCountingTracker();
-        this.ioOpCallbackProvider = NoOpIOOperationCallback.INSTANCE;
+        this.ioOpCallback = NoOpIOOperationCallback.INSTANCE;
         this.numMutableComponents = AccessMethodTestsConfig.LSM_RTREE_NUM_MUTABLE_COMPONENTS;
     }
 
@@ -206,7 +206,7 @@
         return mergePolicy;
     }
 
-    public ILSMIOOperationCallbackProvider getIOOperationCallbackProvider() {
-        return ioOpCallbackProvider;
+    public ILSMIOOperationCallback getIOOperationCallback() {
+        return ioOpCallback;
     }
 }
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
index 069524c..6a0a6bb 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/util/LSMRTreeWithAntiMatterTuplesTestContext.java
@@ -26,7 +26,7 @@
 import edu.uci.ics.hyracks.dataflow.common.util.SerdeUtils;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -72,8 +72,8 @@
             FileReference file, IBufferCache diskBufferCache, IFileMapProvider diskFileMapProvider,
             ISerializerDeserializer[] fieldSerdes, IPrimitiveValueProviderFactory[] valueProviderFactories,
             int numKeyFields, RTreePolicyType rtreePolicyType, ILSMMergePolicy mergePolicy,
-            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
-            ILSMIOOperationCallbackProvider ioOpCallbackProvider) throws Exception {
+            ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallback ioOpCallback)
+            throws Exception {
         ITypeTraits[] typeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
         IBinaryComparatorFactory[] rtreeCmpFactories = SerdeUtils
                 .serdesToComparatorFactories(fieldSerdes, numKeyFields);
@@ -81,7 +81,7 @@
                 fieldSerdes.length);
         LSMRTreeWithAntiMatterTuples lsmTree = LSMRTreeUtils.createLSMTreeWithAntiMatterTuples(virtualBufferCaches,
                 file, diskBufferCache, diskFileMapProvider, typeTraits, rtreeCmpFactories, btreeCmpFactories,
-                valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallbackProvider,
+                valueProviderFactories, rtreePolicyType, mergePolicy, opTracker, ioScheduler, ioOpCallback,
                 LSMRTreeUtils.proposeBestLinearizer(typeTraits, rtreeCmpFactories.length));
         LSMRTreeWithAntiMatterTuplesTestContext testCtx = new LSMRTreeWithAntiMatterTuplesTestContext(fieldSerdes,
                 lsmTree);
diff --git a/pregelix/pregelix-benchmark/pom.xml b/pregelix/pregelix-benchmark/pom.xml
index 4d7d456..c1344ea 100644
--- a/pregelix/pregelix-benchmark/pom.xml
+++ b/pregelix/pregelix-benchmark/pom.xml
@@ -12,6 +12,42 @@
 	<artifactId>pregelix-benchmark</artifactId>
 	<name>pregelix-benchmark</name>
 	<url>http://maven.apache.org</url>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-compiler-plugin</artifactId>
+				<version>2.0.2</version>
+				<configuration>
+					<source>1.7</source>
+					<target>1.7</target>
+					<fork>true</fork>
+				</configuration>
+			</plugin>
+
+			<plugin>
+				<artifactId>maven-assembly-plugin</artifactId>
+				<version>2.2-beta-5</version>
+				<configuration>
+					<descriptorRefs>
+						<descriptorRef>jar-with-dependencies</descriptorRef>
+					</descriptorRefs>
+				</configuration>
+				<executions>
+					<execution>
+						<id>make-my-jar-with-dependencies</id>
+						<phase>package</phase>
+						<goals>
+							<goal>single</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+
+
 	<dependencies>
 		<dependency>
 			<groupId>junit</groupId>
@@ -26,6 +62,12 @@
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
+			<groupId>org.apache.giraph</groupId>
+			<artifactId>giraph-examples</artifactId>
+			<version>1.0.0</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-hdfs-core</artifactId>
 			<version>0.2.10-SNAPSHOT</version>
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java
deleted file mode 100644
index 04c29de..0000000
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Copyright 2009-2013 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.pregelix.benchmark;
-
-import org.apache.giraph.graph.Vertex;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
-import org.apache.hadoop.io.VLongWritable;
-
-/**
- * Demonstrates the basic Pregel PageRank implementation.
- */
-public class PageRankVertex extends Vertex<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
-
-    public static final String ITERATIONS = "HyracksPageRankVertex.iteration";
-    private final DoubleWritable vertexValue = new DoubleWritable();
-    private final DoubleWritable msg = new DoubleWritable();
-    private int maxIteration = -1;
-
-    @Override
-    public void compute(Iterable<DoubleWritable> msgIterator) {
-        if (maxIteration < 0) {
-            maxIteration = getContext().getConfiguration().getInt(ITERATIONS, 10);
-        }
-        if (getSuperstep() == 1) {
-            vertexValue.set(1.0 / getTotalNumVertices());
-        }
-        if (getSuperstep() >= 2 && getSuperstep() <= maxIteration) {
-            double sum = 0;
-            for (DoubleWritable msg : msgIterator) {
-                sum += msg.get();
-            }
-            vertexValue.set((0.15 / getTotalNumVertices()) + 0.85 * sum);
-        }
-
-        if (getSuperstep() >= 1 && getSuperstep() < maxIteration) {
-            long edges = getNumEdges();
-            msg.set(vertexValue.get() / edges);
-            sendMessageToAllEdges(msg);
-        } else {
-            voteToHalt();
-        }
-    }
-
-}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
similarity index 60%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
index 3d85f66..b290907 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCInputFormat.java
@@ -12,7 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.benchmark;
+
+package edu.uci.ics.pregelix.benchmark.io;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -24,14 +25,13 @@
 import org.apache.giraph.edge.Edge;
 import org.apache.giraph.edge.MapMutableEdge;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
-import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextCCInputFormat extends TextVertexInputFormat<LongWritable, LongWritable, NullWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,25 +39,25 @@
             String[] items;
 
             @Override
-            protected VLongWritable getId(Text line) throws IOException {
+            protected LongWritable getId(Text line) throws IOException {
                 items = line.toString().split(" ");
-                return new VLongWritable(Long.parseLong(items[0]));
+                return new LongWritable(Long.parseLong(items[0]));
             }
 
             @Override
-            protected DoubleWritable getValue(Text line) throws IOException {
+            protected LongWritable getValue(Text line) throws IOException {
                 return null;
             }
 
             @Override
-            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
-                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
-                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+            protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+                Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
                 for (int i = 1; i < items.length; i++) {
-                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
                 }
-                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
                     edge.setEntry(entry);
                     edge.setValue(null);
                     edges.add(edge);
@@ -67,4 +67,5 @@
 
         };
     }
+
 }
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCOutputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCOutputFormat.java
new file mode 100644
index 0000000..770c6e1
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextCCOutputFormat.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2013 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.pregelix.benchmark.io;
+
+import java.io.IOException;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextCCOutputFormat extends TextVertexOutputFormat<LongWritable, LongWritable, NullWritable> {
+
+    @Override
+    public TextVertexWriter createVertexWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+        return new TextVertexWriterToEachLine() {
+
+            @Override
+            protected Text convertVertexToLine(Vertex<LongWritable, LongWritable, NullWritable, ?> vertex)
+                    throws IOException {
+                return new Text(vertex.getId() + " " + vertex.getValue());
+            }
+
+        };
+    }
+
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
similarity index 64%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
index 3d85f66..38eef3a 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPRInputFormat.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.benchmark;
+package edu.uci.ics.pregelix.benchmark.io;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -25,13 +25,13 @@
 import org.apache.giraph.edge.MapMutableEdge;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextPRInputFormat extends TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,9 @@
             String[] items;
 
             @Override
-            protected VLongWritable getId(Text line) throws IOException {
+            protected LongWritable getId(Text line) throws IOException {
                 items = line.toString().split(" ");
-                return new VLongWritable(Long.parseLong(items[0]));
+                return new LongWritable(Long.parseLong(items[0]));
             }
 
             @Override
@@ -50,14 +50,14 @@
             }
 
             @Override
-            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
-                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
-                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+            protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+                Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
                 for (int i = 1; i < items.length; i++) {
-                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
                 }
-                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
                     edge.setEntry(entry);
                     edge.setValue(null);
                     edges.add(edge);
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPROutputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPROutputFormat.java
new file mode 100644
index 0000000..b14de6f
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextPROutputFormat.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2013 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.pregelix.benchmark.io;
+
+import java.io.IOException;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.giraph.io.formats.TextVertexOutputFormat;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextPROutputFormat extends TextVertexOutputFormat<LongWritable, DoubleWritable, NullWritable> {
+
+    @Override
+    public TextVertexWriter createVertexWriter(TaskAttemptContext context) throws IOException, InterruptedException {
+        return new TextVertexWriterToEachLine() {
+
+            @Override
+            protected Text convertVertexToLine(Vertex<LongWritable, DoubleWritable, NullWritable, ?> vertex)
+                    throws IOException {
+                return new Text(vertex.getId() + " " + vertex.getValue());
+            }
+
+        };
+    }
+
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
similarity index 65%
rename from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
rename to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
index 3d85f66..953e93c 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io/TextSPInputFormat.java
@@ -12,7 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.benchmark;
+
+package edu.uci.ics.pregelix.benchmark.io;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -25,13 +26,12 @@
 import org.apache.giraph.edge.MapMutableEdge;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextSPInputFormat extends TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,9 @@
             String[] items;
 
             @Override
-            protected VLongWritable getId(Text line) throws IOException {
+            protected LongWritable getId(Text line) throws IOException {
                 items = line.toString().split(" ");
-                return new VLongWritable(Long.parseLong(items[0]));
+                return new LongWritable(Long.parseLong(items[0]));
             }
 
             @Override
@@ -50,16 +50,16 @@
             }
 
             @Override
-            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
-                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
-                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+            protected Iterable<Edge<LongWritable, DoubleWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, DoubleWritable>> edges = new ArrayList<Edge<LongWritable, DoubleWritable>>();
+                Map<LongWritable, DoubleWritable> edgeMap = new HashMap<LongWritable, DoubleWritable>();
                 for (int i = 1; i < items.length; i++) {
-                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
                 }
-                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                for (Entry<LongWritable, DoubleWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, DoubleWritable> edge = new MapMutableEdge<LongWritable, DoubleWritable>();
                     edge.setEntry(entry);
-                    edge.setValue(null);
+                    edge.setValue(new DoubleWritable(1.0));
                     edges.add(edge);
                 }
                 return edges;
@@ -67,4 +67,5 @@
 
         };
     }
+
 }
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextCCInputFormat2.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextCCInputFormat2.java
new file mode 100644
index 0000000..0a70b3c
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextCCInputFormat2.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2013 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.pregelix.benchmark.io2;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.MapMutableEdge;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextCCInputFormat2 extends TextVertexInputFormat<LongWritable, LongWritable, NullWritable> {
+
+    @Override
+    public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
+        return new TextVertexReaderFromEachLine() {
+            String[] items;
+
+            @Override
+            protected LongWritable getId(Text line) throws IOException {
+                String[] kv = line.toString().split("\t");
+                items = kv[1].split(" ");
+                return new LongWritable(Long.parseLong(kv[0]));
+            }
+
+            @Override
+            protected LongWritable getValue(Text line) throws IOException {
+                return null;
+            }
+
+            @Override
+            protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+                Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
+                for (int i = 1; i < items.length; i++) {
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
+                }
+                for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
+                    edge.setEntry(entry);
+                    edge.setValue(null);
+                    edges.add(edge);
+                }
+                return edges;
+            }
+
+        };
+    }
+
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
similarity index 62%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
index 3d85f66..63a4519 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextPRInputFormat2.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.benchmark;
+package edu.uci.ics.pregelix.benchmark.io2;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -25,13 +25,13 @@
 import org.apache.giraph.edge.MapMutableEdge;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextPRInputFormat2 extends TextVertexInputFormat<LongWritable, DoubleWritable, NullWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,10 @@
             String[] items;
 
             @Override
-            protected VLongWritable getId(Text line) throws IOException {
-                items = line.toString().split(" ");
-                return new VLongWritable(Long.parseLong(items[0]));
+            protected LongWritable getId(Text line) throws IOException {
+                String[] kv = line.toString().split("\t");
+                items = kv[1].split(" ");
+                return new LongWritable(Long.parseLong(items[0]));
             }
 
             @Override
@@ -50,14 +51,14 @@
             }
 
             @Override
-            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
-                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
-                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+            protected Iterable<Edge<LongWritable, NullWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, NullWritable>> edges = new ArrayList<Edge<LongWritable, NullWritable>>();
+                Map<LongWritable, NullWritable> edgeMap = new HashMap<LongWritable, NullWritable>();
                 for (int i = 1; i < items.length; i++) {
-                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
                 }
-                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                for (Entry<LongWritable, NullWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, NullWritable> edge = new MapMutableEdge<LongWritable, NullWritable>();
                     edge.setEntry(entry);
                     edge.setValue(null);
                     edges.add(edge);
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
similarity index 63%
copy from pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
copy to pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
index 3d85f66..fdb1061 100644
--- a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/io2/TextSPInputFormat2.java
@@ -12,7 +12,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.pregelix.benchmark;
+
+package edu.uci.ics.pregelix.benchmark.io2;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -25,13 +26,12 @@
 import org.apache.giraph.edge.MapMutableEdge;
 import org.apache.giraph.io.formats.TextVertexInputFormat;
 import org.apache.hadoop.io.DoubleWritable;
-import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.VLongWritable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+public class TextSPInputFormat2 extends TextVertexInputFormat<LongWritable, DoubleWritable, DoubleWritable> {
 
     @Override
     public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
@@ -39,9 +39,10 @@
             String[] items;
 
             @Override
-            protected VLongWritable getId(Text line) throws IOException {
-                items = line.toString().split(" ");
-                return new VLongWritable(Long.parseLong(items[0]));
+            protected LongWritable getId(Text line) throws IOException {
+                String[] kv = line.toString().split("\t");
+                items = kv[1].split(" ");
+                return new LongWritable(Long.parseLong(kv[0]));
             }
 
             @Override
@@ -50,16 +51,16 @@
             }
 
             @Override
-            protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
-                List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
-                Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+            protected Iterable<Edge<LongWritable, DoubleWritable>> getEdges(Text line) throws IOException {
+                List<Edge<LongWritable, DoubleWritable>> edges = new ArrayList<Edge<LongWritable, DoubleWritable>>();
+                Map<LongWritable, DoubleWritable> edgeMap = new HashMap<LongWritable, DoubleWritable>();
                 for (int i = 1; i < items.length; i++) {
-                    edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+                    edgeMap.put(new LongWritable(Long.parseLong(items[i])), null);
                 }
-                for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
-                    MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+                for (Entry<LongWritable, DoubleWritable> entry : edgeMap.entrySet()) {
+                    MapMutableEdge<LongWritable, DoubleWritable> edge = new MapMutableEdge<LongWritable, DoubleWritable>();
                     edge.setEntry(entry);
-                    edge.setValue(null);
+                    edge.setValue(new DoubleWritable(1.0));
                     edges.add(edge);
                 }
                 return edges;
@@ -67,4 +68,5 @@
 
         };
     }
+
 }
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ConnectedComponentsVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ConnectedComponentsVertex.java
new file mode 100644
index 0000000..3789d6d
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ConnectedComponentsVertex.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2009-2013 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.pregelix.benchmark.vertex;
+
+import java.io.IOException;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+
+public class ConnectedComponentsVertex extends Vertex<LongWritable, LongWritable, NullWritable, LongWritable> {
+    /**
+     * Propagates the smallest vertex id to all neighbors. Will always choose to
+     * halt and only reactivate if a smaller id has been sent to it.
+     * 
+     * @param messages
+     *            Iterator of messages from the previous superstep.
+     * @throws IOException
+     */
+    @Override
+    public void compute(Iterable<LongWritable> messages) throws IOException {
+        long currentComponent = getValue().get();
+
+        // First superstep is special, because we can simply look at the neighbors
+        if (getSuperstep() == 0) {
+            for (Edge<LongWritable, NullWritable> edge : getEdges()) {
+                long neighbor = edge.getTargetVertexId().get();
+                if (neighbor < currentComponent) {
+                    currentComponent = neighbor;
+                }
+            }
+            // Only need to send value if it is not the own id
+            if (currentComponent != getValue().get()) {
+                setValue(new LongWritable(currentComponent));
+                for (Edge<LongWritable, NullWritable> edge : getEdges()) {
+                    LongWritable neighbor = edge.getTargetVertexId();
+                    if (neighbor.get() > currentComponent) {
+                        sendMessage(neighbor, getValue());
+                    }
+                }
+            }
+
+            voteToHalt();
+            return;
+        }
+
+        boolean changed = false;
+        // did we get a smaller id ?
+        for (LongWritable message : messages) {
+            long candidateComponent = message.get();
+            if (candidateComponent < currentComponent) {
+                currentComponent = candidateComponent;
+                changed = true;
+            }
+        }
+
+        // propagate new component id to the neighbors
+        if (changed) {
+            setValue(new LongWritable(currentComponent));
+            sendMessageToAllEdges(getValue());
+        }
+        voteToHalt();
+    }
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/PageRankVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/PageRankVertex.java
new file mode 100644
index 0000000..86e90dd
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/PageRankVertex.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2013 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.pregelix.benchmark.vertex;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.examples.RandomWalkVertex;
+import org.apache.giraph.utils.MathUtils;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+
+/**
+ * The PageRank algorithm, with uniform transition probabilities on the edges
+ * http://en.wikipedia.org/wiki/PageRank
+ */
+public class PageRankVertex extends RandomWalkVertex<NullWritable> {
+
+    @Override
+    protected double transitionProbability(double stateProbability, Edge<LongWritable, NullWritable> edge) {
+        return stateProbability / getNumEdges();
+    }
+
+    @Override
+    protected double recompute(Iterable<DoubleWritable> partialRanks, double teleportationProbability) {
+
+        // rank contribution from incident neighbors
+        double rankFromNeighbors = MathUtils.sum(partialRanks);
+        // rank contribution from dangling vertices
+        double danglingContribution = getDanglingProbability() / getTotalNumVertices();
+
+        // recompute rank
+        return (1d - teleportationProbability) * (rankFromNeighbors + danglingContribution) + teleportationProbability
+                / getTotalNumVertices();
+    }
+}
diff --git a/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ShortestPathsVertex.java b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ShortestPathsVertex.java
new file mode 100644
index 0000000..755a3d0
--- /dev/null
+++ b/pregelix/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/vertex/ShortestPathsVertex.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.pregelix.benchmark.vertex;
+
+import java.io.IOException;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.LongWritable;
+
+/**
+ * Shortest paths algorithm.
+ */
+public class ShortestPathsVertex extends Vertex<LongWritable, DoubleWritable, DoubleWritable, DoubleWritable> {
+    /** Source id. */
+    public static final String SOURCE_ID = "giraph.shortestPathsBenchmark.sourceId";
+    /** Default source id. */
+    public static final long SOURCE_ID_DEFAULT = 1;
+
+    private boolean isSource() {
+        return getId().get() == getConf().getLong(SOURCE_ID, SOURCE_ID_DEFAULT);
+    }
+
+    @Override
+    public void compute(Iterable<DoubleWritable> messages) throws IOException {
+        if (getSuperstep() == 0) {
+            setValue(new DoubleWritable(Double.MAX_VALUE));
+        }
+
+        double minDist = isSource() ? 0d : Double.MAX_VALUE;
+        for (DoubleWritable message : messages) {
+            minDist = Math.min(minDist, message.get());
+        }
+
+        if (minDist < getValue().get()) {
+            setValue(new DoubleWritable(minDist));
+            for (Edge<LongWritable, DoubleWritable> edge : getEdges()) {
+                double distance = minDist + edge.getValue().get();
+                sendMessage(edge.getTargetVertexId(), new DoubleWritable(distance));
+            }
+        }
+
+        voteToHalt();
+    }
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
index 4863378..88edc91 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
@@ -63,6 +63,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSReadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
@@ -95,7 +96,8 @@
 import edu.uci.ics.pregelix.core.data.TypeTraits;
 import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawNormalizedKeyComputerFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ClearStateOperatorDescriptor;
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -196,17 +198,14 @@
         return jobId;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     public JobSpecification generateCreatingJob() throws HyracksException {
-        Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
         JobSpecification spec = new JobSpecification();
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
         typeTraits[1] = new TypeTraits(false);
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
 
         int[] keyFields = new int[1];
         keyFields[0] = 0;
@@ -264,8 +263,7 @@
         sortFields[0] = 0;
         INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameLimit, sortFields,
                 nkmFactory, comparatorFactories, recordDescriptor);
         ClusterConfig.setLocationConstraint(spec, sorter);
@@ -291,12 +289,12 @@
         ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
         spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
-                comparatorFactories), sorter, 0, writer, 0);
+                comparatorFactories, nkmFactory), sorter, 0, writer, 0);
         spec.setFrameSize(frameSize);
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
+    @SuppressWarnings({ "rawtypes" })
     public JobSpecification scanIndexPrintGraph(String nodeName, String path) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
         Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
@@ -324,8 +322,7 @@
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
@@ -355,16 +352,17 @@
          */
         int[] sortFields = new int[1];
         sortFields[0] = 0;
+        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
         ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, hashPartitionComputerFactory, sortFields,
-                comparatorFactories), scanner, 0, writer, 0);
+                comparatorFactories, nkmFactory), scanner, 0, writer, 0);
         spec.setFrameSize(frameSize);
         return spec;
     }
 
     public JobSpecification scanIndexWriteGraph() throws HyracksException {
-        JobSpecification spec = scanIndexWriteToHDFS(conf);
+        JobSpecification spec = scanIndexWriteToHDFS(conf, false);
         return spec;
     }
 
@@ -380,7 +378,7 @@
             FileSystem dfs = FileSystem.get(tmpJob.getConfiguration());
 
             dfs.delete(new Path(BspUtils.getVertexCheckpointPath(conf, lastSuccessfulIteration)), true);
-            JobSpecification vertexCkpSpec = scanIndexWriteToHDFS(tmpJob.getConfiguration());
+            JobSpecification vertexCkpSpec = scanIndexWriteToHDFS(tmpJob.getConfiguration(), true);
 
             dfs.delete(new Path(BspUtils.getMessageCheckpointPath(conf, lastSuccessfulIteration)), true);
             JobSpecification[] stateCkpSpecs = generateStateCheckpointing(lastSuccessfulIteration);
@@ -523,10 +521,9 @@
          */
         int[] sortFields = new int[1];
         sortFields[0] = 0;
-        INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
+        INormalizedKeyComputerFactory nkmFactory = RawNormalizedKeyComputerFactory.INSTANCE;
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, sortFields,
                 nkmFactory, comparatorFactories, recordDescriptor);
         ClusterConfig.setLocationConstraint(spec, sorter);
@@ -556,8 +553,9 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
-    private JobSpecification scanIndexWriteToHDFS(Configuration conf) throws HyracksDataException, HyracksException {
+    @SuppressWarnings({ "rawtypes" })
+    private JobSpecification scanIndexWriteToHDFS(Configuration conf, boolean ckpointing) throws HyracksDataException,
+            HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
         Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
         JobSpecification spec = new JobSpecification();
@@ -584,8 +582,7 @@
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
@@ -597,6 +594,18 @@
                 null, null, true, true, getIndexDataflowHelperFactory(), false, NoOpOperationCallbackFactory.INSTANCE);
         ClusterConfig.setLocationConstraint(spec, scanner);
 
+        ExternalSortOperatorDescriptor sort = null;
+        if (!ckpointing) {
+            int[] keyFields = new int[] { 0 };
+            INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getFinalNormalizedKeyComputerFactory(conf);
+            IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
+            sortCmpFactories[0] = JobGenUtil.getFinalBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
+                    .getClass());
+            sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields, nkmFactory, sortCmpFactories,
+                    recordDescriptor);
+            ClusterConfig.setLocationConstraint(spec, scanner);
+        }
+
         /**
          * construct write file operator
          */
@@ -610,7 +619,12 @@
          * connect operator descriptors
          */
         spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, scanner, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, writer, 0);
+        if (!ckpointing) {
+            spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, sort, 0);
+            spec.connect(new OneToOneConnectorDescriptor(spec), sort, 0, writer, 0);
+        } else {
+            spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, writer, 0);
+        }
         spec.setFrameSize(frameSize);
         return spec;
     }
@@ -715,7 +729,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastCheckpointedIteration,
                 WritableComparator.get(vertexIdClass).getClass());
         ExternalSortOperatorDescriptor sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, recordDescriptor);
+                nkmFactory, sortCmpFactories, recordDescriptor, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, sort);
 
         /**
@@ -739,8 +753,7 @@
          * connect operator descriptors
          */
         ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
-        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0,
-                sort, 0);
+        spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0, sort, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), sort, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
index 9389f62..103c1b6 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
@@ -50,6 +50,7 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSReadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -62,7 +63,7 @@
 import edu.uci.ics.pregelix.core.data.TypeTraits;
 import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -97,7 +98,6 @@
         super(job);
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
         Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
@@ -129,8 +129,7 @@
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
@@ -195,7 +194,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -282,8 +281,8 @@
 
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), scanner, 5, btreeBulkLoad, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
-        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
-                localGby, 0, globalGby, 0);
+        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+                nkmFactory), localGby, 0, globalGby, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
@@ -300,7 +299,6 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -316,8 +314,7 @@
         RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), messageValueClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
                 MsgList.class.getName());
         RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
@@ -401,7 +398,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -503,8 +500,8 @@
 
         spec.connect(new MToNPartitioningConnectorDescriptor(spec, partionFactory), join, 5, btreeBulkLoad, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
-        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
-                localGby, 0, globalGby, 0);
+        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+                nkmFactory), localGby, 0, globalGby, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
@@ -601,7 +598,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastSuccessfulIteration,
                 WritableComparator.get(vertexIdClass).getClass());
         ExternalSortOperatorDescriptor sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, recordDescriptor);
+                nkmFactory, sortCmpFactories, recordDescriptor, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, sort);
 
         /**
@@ -633,7 +630,7 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
+    @SuppressWarnings({ "rawtypes" })
     private JobSpecification generateSecondaryBTreeCheckpoint(int lastSuccessfulIteration, PregelixJob job)
             throws HyracksException {
         job.setOutputFormatClass(SequenceFileOutputFormat.class);
@@ -668,8 +665,7 @@
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), msgListClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
         typeTraits[0] = new TypeTraits(false);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
index 287b797..308f422 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
@@ -32,6 +32,7 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -42,7 +43,7 @@
 import edu.uci.ics.pregelix.core.data.TypeTraits;
 import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -74,7 +75,6 @@
         super(job);
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -101,8 +101,7 @@
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
@@ -138,7 +137,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -241,8 +240,8 @@
          * connect the group-by operator
          */
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
-        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
-                localGby, 0, globalGby, 0);
+        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+                nkmFactory), localGby, 0, globalGby, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink2, 0);
@@ -258,7 +257,6 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -274,8 +272,7 @@
         RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), messageValueClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
                 MsgList.class.getName());
         RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
@@ -338,7 +335,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -440,8 +437,8 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), deleteOp, 0, emptySink4, 0);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), localSort, 0, localGby, 0);
-        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories),
-                localGby, 0, globalGby, 0);
+        spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, partionFactory, keyFields, sortCmpFactories,
+                nkmFactory), localGby, 0, globalGby, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), globalGby, 0, materialize, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
         spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
index 3b3c9e7..718a271 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
@@ -31,6 +31,7 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -41,7 +42,7 @@
 import edu.uci.ics.pregelix.core.data.TypeTraits;
 import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -73,7 +74,6 @@
         super(job);
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -100,8 +100,7 @@
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
@@ -140,7 +139,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
         /**
@@ -247,7 +246,6 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -263,8 +261,7 @@
         RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), messageValueClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
                 MsgList.class.getName());
         RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
@@ -327,7 +324,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
         /**
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
index e334095..75635c9 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
@@ -31,6 +31,7 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
@@ -41,7 +42,7 @@
 import edu.uci.ics.pregelix.core.data.TypeTraits;
 import edu.uci.ics.pregelix.core.hadoop.config.ConfigurationFactory;
 import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
-import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
 import edu.uci.ics.pregelix.core.util.DataflowUtils;
 import edu.uci.ics.pregelix.dataflow.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
@@ -73,7 +74,6 @@
         super(job);
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -100,8 +100,7 @@
         RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), vertexClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX);
 
         ITypeTraits[] typeTraits = new ITypeTraits[2];
@@ -137,7 +136,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -153,7 +152,7 @@
          * construct global sort operator
          */
         ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
         /**
@@ -261,7 +260,6 @@
         return spec;
     }
 
-    @SuppressWarnings({ "rawtypes", "unchecked" })
     @Override
     protected JobSpecification generateNonFirstIteration(int iteration) throws HyracksException {
         Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
@@ -277,8 +275,7 @@
         RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
                 vertexIdClass.getName(), messageValueClass.getName());
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
-        comparatorFactories[0] = new WritableComparingBinaryComparatorFactory(WritableComparator.get(vertexIdClass)
-                .getClass());
+        comparatorFactories[0] = RawBinaryComparatorFactory.INSTANCE;
         RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
                 MsgList.class.getName());
         RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(),
@@ -341,7 +338,7 @@
         sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, WritableComparator.get(vertexIdClass)
                 .getClass());
         ExternalSortOperatorDescriptor localSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, localSort);
 
         /**
@@ -357,7 +354,7 @@
          * construct global sort operator
          */
         ExternalSortOperatorDescriptor globalSort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
-                nkmFactory, sortCmpFactories, rdUnnestedMessage);
+                nkmFactory, sortCmpFactories, rdUnnestedMessage, Algorithm.QUICK_SORT);
         ClusterConfig.setLocationConstraint(spec, globalSort);
 
         /**
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java
index c4f54fe..97cea99 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenUtil.java
@@ -21,10 +21,12 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
 import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawBinaryComparatorFactory;
+import edu.uci.ics.pregelix.core.runtime.touchpoint.RawNormalizedKeyComputerFactory;
 import edu.uci.ics.pregelix.core.runtime.touchpoint.WritableComparingBinaryComparatorFactory;
 import edu.uci.ics.pregelix.runtime.touchpoint.VertexIdNormalizedKeyComputerFactory;
 
-@SuppressWarnings({ "rawtypes", "unchecked" })
+@SuppressWarnings({ "rawtypes" })
 public class JobGenUtil {
 
     /**
@@ -36,11 +38,7 @@
      * @return
      */
     public static INormalizedKeyComputerFactory getINormalizedKeyComputerFactory(Configuration conf) {
-        Class<? extends NormalizedKeyComputer> clazz = BspUtils.getNormalizedKeyComputerClass(conf);
-        if (clazz.equals(NormalizedKeyComputer.class)) {
-            return null;
-        }
-        return new VertexIdNormalizedKeyComputerFactory(clazz);
+        return RawNormalizedKeyComputerFactory.INSTANCE;
     }
 
     /**
@@ -52,6 +50,33 @@
      * @return
      */
     public static IBinaryComparatorFactory getIBinaryComparatorFactory(int iteration, Class keyClass) {
+        return RawBinaryComparatorFactory.INSTANCE;
+    }
+
+    /**
+     * get normalized key factory for the final output job
+     * 
+     * @param iteration
+     * @param keyClass
+     * @return
+     */
+    public static INormalizedKeyComputerFactory getFinalNormalizedKeyComputerFactory(Configuration conf) {
+        Class<? extends NormalizedKeyComputer> clazz = BspUtils.getNormalizedKeyComputerClass(conf);
+        if (clazz.equals(NormalizedKeyComputer.class)) {
+            return null;
+        }
+        return new VertexIdNormalizedKeyComputerFactory(clazz);
+    }
+
+    /**
+     * get comparator for the final output job
+     * 
+     * @param iteration
+     * @param keyClass
+     * @return
+     */
+    @SuppressWarnings("unchecked")
+    public static IBinaryComparatorFactory getFinalBinaryComparatorFactory(Class keyClass) {
         return new WritableComparingBinaryComparatorFactory(keyClass);
     }
 
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawBinaryComparatorFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawBinaryComparatorFactory.java
new file mode 100644
index 0000000..c0173ed
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawBinaryComparatorFactory.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2013 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.pregelix.core.runtime.touchpoint;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class RawBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static IBinaryComparatorFactory INSTANCE = new RawBinaryComparatorFactory();
+
+    private RawBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int commonLength = Math.min(l1, l2);
+                for (int i = 0; i < commonLength; i++) {
+                    if (b1[s1 + i] != b2[s2 + i]) {
+                        return (b1[s1 + i] & 0xff) - (b2[s2 + i] & 0xff);
+                    }
+                }
+                int difference = l1 - l2;
+                return difference == 0 ? 0 : (difference > 0 ? 1 : -1);
+            }
+
+        };
+    }
+}
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawNormalizedKeyComputerFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..6aa4af9
--- /dev/null
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/RawNormalizedKeyComputerFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2013 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.pregelix.core.runtime.touchpoint;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+
+public class RawNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+    private static final long serialVersionUID = 1L;
+    public static final INormalizedKeyComputerFactory INSTANCE = new RawNormalizedKeyComputerFactory();
+
+    private RawNormalizedKeyComputerFactory() {
+
+    }
+
+    @Override
+    public INormalizedKeyComputer createNormalizedKeyComputer() {
+        return new INormalizedKeyComputer() {
+
+            @Override
+            public int normalize(byte[] bytes, int start, int length) {
+                int nk = 0;
+                for (int i = 0; i < 4; i++) {
+                    nk <<= 8;
+                    if (i < length) {
+                        nk += (bytes[start + i] & 0xff);
+                    }
+                }
+                return nk;
+            }
+
+        };
+    }
+}
diff --git a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
index 4c7f91d..f599996 100644
--- a/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
+++ b/pregelix/pregelix-core/src/test/java/edu/uci/ics/pregelix/core/join/JoinTest.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -34,6 +35,7 @@
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
@@ -89,6 +91,7 @@
             UTF8StringPointable.FACTORY);
     private IBinaryComparatorFactory stringComparatorFactory = new PointableBinaryComparatorFactory(
             UTF8StringPointable.FACTORY);
+    private INormalizedKeyComputerFactory nmkFactory = new UTF8StringNormalizedKeyComputerFactory();
 
     private void cleanupStores() throws IOException {
         FileUtils.forceMkdir(new File("teststore"));
@@ -211,7 +214,8 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), join, 0, sorter, 0);
         IConnectorDescriptor joinWriterConn = new MToNPartitioningMergingConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories);
+                        new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+                nmkFactory);
         spec.connect(joinWriterConn, sorter, 0, writer, 0);
 
         spec.addRoot(writer);
@@ -284,8 +288,8 @@
 
         spec.connect(new OneToOneConnectorDescriptor(spec), custScanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
-                sortFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories),
-                sorter, 0, writer, 0);
+                sortFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+                nmkFactory), sorter, 0, writer, 0);
 
         spec.addRoot(writer);
         runTest(spec);
@@ -368,11 +372,11 @@
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
-                keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories),
-                sorter, 0, join, 0);
+                keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+                nmkFactory), sorter, 0, join, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
-                keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories),
-                join, 0, writer, 0);
+                keyFields, new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+                nmkFactory), join, 0, writer, 0);
 
         spec.addRoot(writer);
         runTest(spec);
@@ -477,7 +481,8 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), project, 0, sorter, 0);
         IConnectorDescriptor joinWriterConn = new MToNPartitioningMergingConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(new int[] { 9 },
-                        new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories);
+                        new IBinaryHashFunctionFactory[] { stringHashFactory }), sortFields, comparatorFactories,
+                nmkFactory);
         spec.connect(joinWriterConn, sorter, 0, writer, 0);
 
         spec.addRoot(writer);
@@ -573,7 +578,8 @@
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
                 keyFields, new IBinaryHashFunctionFactory[] { new PointableBinaryHashFunctionFactory(
-                        UTF8StringPointable.FACTORY) }), sortFields, comparatorFactories), sorter, 0, join, 0);
+                        UTF8StringPointable.FACTORY) }), sortFields, comparatorFactories, nmkFactory), sorter, 0, join,
+                0);
 
         IBinaryComparatorFactory[] mergeComparatorFactories = new IBinaryComparatorFactory[2];
         mergeComparatorFactories[0] = new PointableBinaryComparatorFactory(UTF8StringPointable.FACTORY);
@@ -581,7 +587,8 @@
         int[] mergeFields = new int[] { 9, 0 };
         spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
                 new int[] { 9 }, new IBinaryHashFunctionFactory[] { new PointableBinaryHashFunctionFactory(
-                        UTF8StringPointable.FACTORY) }), mergeFields, comparatorFactories), join, 0, writer, 0);
+                        UTF8StringPointable.FACTORY) }), mergeFields, comparatorFactories, nmkFactory), join, 0,
+                writer, 0);
 
         spec.addRoot(writer);
         runTest(spec);