Implemented the memory-bounded HashGroupby and HashJoin for BigObject

It contains both hash grouby and hash join changes.

The main change is
1. update the ExternalGroupby to Hash-based groupby
2. update the Join operators to use the Buffermanager.

The buffer manager part is moved from the Sort package to upper
level so that it can be shared by all the operators.

Change-Id: I248f3a374fdacad7d57e49cf18d8233745e55460
Reviewed-on: https://asterix-gerrit.ics.uci.edu/398
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
index efb9681..b15ea0b 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
@@ -24,7 +24,6 @@
 import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.ListSet;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -59,26 +58,26 @@
 import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.operators.aggreg.SerializableAggregatorDescriptorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
 import org.apache.hyracks.dataflow.std.group.HashSpillableTableFactory;
 import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
 
 public class ExternalGroupByPOperator extends AbstractPhysicalOperator {
 
-    private int tableSize = 0;
-    private int frameLimit = 0;
+    private final int tableSize;
+    private final long fileSize;
+    private final int frameLimit;
     private List<LogicalVariable> columnSet = new ArrayList<LogicalVariable>();
 
     public ExternalGroupByPOperator(List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> gbyList, int frameLimit,
-            int tableSize) {
+            int tableSize, long fileSize) {
         this.tableSize = tableSize;
         this.frameLimit = frameLimit;
+        this.fileSize = fileSize;
         computeColumnSet(gbyList);
     }
 
@@ -140,8 +139,8 @@
         AbstractLogicalOperator aop = (AbstractLogicalOperator) op;
         if (aop.getExecutionMode() == ExecutionMode.PARTITIONED) {
             StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
-            pv[0] = new StructuralPropertiesVector(new UnorderedPartitionedProperty(new ListSet<LogicalVariable>(
-                    columnSet), null), null);
+            pv[0] = new StructuralPropertiesVector(
+                    new UnorderedPartitionedProperty(new ListSet<LogicalVariable>(columnSet), null), null);
             return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
         } else {
             return emptyUnaryRequirements();
@@ -151,7 +150,7 @@
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         List<LogicalVariable> gbyCols = getGbyColumns();
         int keys[] = JobGenHelper.variablesToFieldIndexes(gbyCols, inputSchemas[0]);
         GroupByOperator gby = (GroupByOperator) op;
@@ -194,8 +193,8 @@
             AggregateFunctionCallExpression aggFun = (AggregateFunctionCallExpression) exprRef.getValue();
             aff[i++] = expressionRuntimeProvider.createSerializableAggregateFunctionFactory(aggFun, aggOpInputEnv,
                     inputSchemas, context);
-            intermediateTypes.add(partialAggregationTypeComputer.getType(aggFun, aggOpInputEnv,
-                    context.getMetadataProvider()));
+            intermediateTypes
+                    .add(partialAggregationTypeComputer.getType(aggFun, aggOpInputEnv, context.getMetadataProvider()));
         }
 
         int[] keyAndDecFields = new int[keys.length + fdColumns.length];
@@ -219,9 +218,10 @@
         IOperatorDescriptorRegistry spec = builder.getJobSpec();
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(gbyCols,
                 aggOpInputEnv, context);
-        RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
-        IBinaryHashFunctionFactory[] hashFunctionFactories = JobGenHelper.variablesToBinaryHashFunctionFactories(
-                gbyCols, aggOpInputEnv, context);
+        RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema,
+                context);
+        IBinaryHashFunctionFamily[] hashFunctionFactories = JobGenHelper.variablesToBinaryHashFunctionFamilies(gbyCols,
+                aggOpInputEnv, context);
 
         ISerializedAggregateEvaluatorFactory[] merges = new ISerializedAggregateEvaluatorFactory[n];
         List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
@@ -249,13 +249,11 @@
         IAggregatorDescriptorFactory aggregatorFactory = new SerializableAggregatorDescriptorFactory(aff);
         IAggregatorDescriptorFactory mergeFactory = new SerializableAggregatorDescriptorFactory(merges);
 
-        ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories);
-        INormalizedKeyComputerFactory normalizedKeyFactory = JobGenHelper.variablesToAscNormalizedKeyComputerFactory(
-                gbyCols, aggOpInputEnv, context);
-        ExternalGroupOperatorDescriptor gbyOpDesc = new ExternalGroupOperatorDescriptor(spec, keyAndDecFields,
-                frameLimit, comparatorFactories, normalizedKeyFactory, aggregatorFactory, mergeFactory,
-                recordDescriptor, new HashSpillableTableFactory(tpcf, tableSize), false);
-
+        INormalizedKeyComputerFactory normalizedKeyFactory = JobGenHelper
+                .variablesToAscNormalizedKeyComputerFactory(gbyCols, aggOpInputEnv, context);
+        ExternalGroupOperatorDescriptor gbyOpDesc = new ExternalGroupOperatorDescriptor(spec, tableSize, fileSize,
+                keyAndDecFields, frameLimit, comparatorFactories, normalizedKeyFactory, aggregatorFactory, mergeFactory,
+                recordDescriptor, recordDescriptor, new HashSpillableTableFactory(hashFunctionFactories));
         contributeOpDesc(builder, gby, gbyOpDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
index 071ee72..a24b87b 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
@@ -150,7 +150,7 @@
                     case INNER: {
                         opDesc = new HybridHashJoinOperatorDescriptor(spec, getMemSizeInFrames(),
                                 maxInputBuildSizeInFrames, aveRecordsPerFrame, getFudgeFactor(), keysLeft, keysRight,
-                                hashFunFactories, comparatorFactories, recDescriptor, predEvaluatorFactory);
+                                hashFunFactories, comparatorFactories, recDescriptor, predEvaluatorFactory, false, null);
                         break;
                     }
                     case LEFT_OUTER: {
diff --git a/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index 73bba8f..8e12ece 100644
--- a/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -166,7 +166,9 @@
                                     ExternalGroupByPOperator externalGby = new ExternalGroupByPOperator(
                                             gby.getGroupByList(),
                                             physicalOptimizationConfig.getMaxFramesExternalGroupBy(),
-                                            physicalOptimizationConfig.getExternalGroupByTableSize());
+                                            physicalOptimizationConfig.getExternalGroupByTableSize(),
+                                            (long) physicalOptimizationConfig.getMaxFramesExternalGroupBy()
+                                                    * physicalOptimizationConfig.getFrameSize());
                                     op.setPhysicalOperator(externalGby);
                                     break;
                                 }
diff --git a/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java b/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
index 9f0960d..974a079 100644
--- a/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
@@ -90,15 +90,13 @@
             @Override
             public void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor,
                     int stateTupleIndex, AggregateState state) throws HyracksDataException {
-                // it only works if the output of the aggregator fits in one
-                // frame
                 for (int i = 0; i < pipelines.length; i++) {
                     pipelines[i].writeTuple(accessor.getBuffer(), tIndex);
                 }
             }
 
             @Override
-            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor stateAccessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 for (int i = 0; i < pipelines.length; i++) {
                     outputWriter.setInputIdx(i);
diff --git a/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java b/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
index fe42878..6700fb8 100644
--- a/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
@@ -113,12 +113,12 @@
             }
 
             @Override
-            public boolean outputPartialResult(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputPartialResult(ArrayTupleBuilder tb, IFrameTupleAccessor stateAccessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
-                byte[] data = accessor.getBuffer().array();
-                int startOffset = accessor.getTupleStartOffset(tIndex);
-                int aggFieldOffset = accessor.getFieldStartOffset(tIndex, offsetFieldIndex);
-                int refOffset = startOffset + accessor.getFieldSlotsLength() + aggFieldOffset;
+                byte[] data = stateAccessor.getBuffer().array();
+                int startOffset = stateAccessor.getTupleStartOffset(tIndex);
+                int aggFieldOffset = stateAccessor.getFieldStartOffset(tIndex, offsetFieldIndex);
+                int refOffset = startOffset + stateAccessor.getFieldSlotsLength() + aggFieldOffset;
                 int start = refOffset;
                 for (int i = 0; i < aggs.length; i++) {
                     try {
@@ -133,12 +133,12 @@
             }
 
             @Override
-            public boolean outputFinalResult(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputFinalResult(ArrayTupleBuilder tb, IFrameTupleAccessor stateAccessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
-                byte[] data = accessor.getBuffer().array();
-                int startOffset = accessor.getTupleStartOffset(tIndex);
-                int aggFieldOffset = accessor.getFieldStartOffset(tIndex, offsetFieldIndex);
-                int refOffset = startOffset + accessor.getFieldSlotsLength() + aggFieldOffset;
+                byte[] data = stateAccessor.getBuffer().array();
+                int startOffset = stateAccessor.getTupleStartOffset(tIndex);
+                int aggFieldOffset = stateAccessor.getFieldStartOffset(tIndex, offsetFieldIndex);
+                int refOffset = startOffset + stateAccessor.getFieldSlotsLength() + aggFieldOffset;
                 int start = refOffset;
                 for (int i = 0; i < aggs.length; i++) {
                     try {
diff --git a/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java b/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
index 34b4865..13e3646 100644
--- a/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
@@ -90,8 +90,8 @@
             }
 
             @Override
-            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
-                    AggregateState state) throws HyracksDataException {
+            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor stateAccessor,
+                    int tIndex, AggregateState state) throws HyracksDataException {
                 IAggregateEvaluator[] agg = (IAggregateEvaluator[]) state.state;
                 for (int i = 0; i < agg.length; i++) {
                     try {
@@ -123,7 +123,7 @@
             }
 
             @Override
-            public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor,
+            public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor stateAccessor,
                     int tIndex, AggregateState state) throws HyracksDataException {
                 IAggregateEvaluator[] agg = (IAggregateEvaluator[]) state.state;
                 for (int i = 0; i < agg.length; i++) {
diff --git a/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
index bca301f..df05d50 100644
--- a/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
@@ -28,11 +28,12 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
+import org.apache.hyracks.dataflow.std.buffermanager.FrameFreeSlotPolicyFactory;
+import org.apache.hyracks.dataflow.std.buffermanager.IFrameBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFrameMemoryManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFramePool;
 import org.apache.hyracks.dataflow.std.sort.FrameSorterMergeSort;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotLastFit;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
 
 public class InMemorySortRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
@@ -67,7 +68,8 @@
                 writer.open();
                 if (frameSorter == null) {
                     IFrameBufferManager manager = new VariableFrameMemoryManager(
-                            new VariableFramePool(ctx, VariableFramePool.UNLIMITED_MEMORY), new FrameFreeSlotLastFit());
+                            new VariableFramePool(ctx, VariableFramePool.UNLIMITED_MEMORY),
+                            FrameFreeSlotPolicyFactory.createFreeSlotPolicy(EnumFreeSlotPolicy.LAST_FIT));
                     frameSorter = new FrameSorterMergeSort(ctx, manager, sortFields, firstKeyNormalizerFactory,
                             comparatorFactories, outputRecordDesc);
                 }
diff --git a/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java b/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
index 7fcab17..0dc1961 100644
--- a/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
+++ b/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
@@ -46,7 +46,6 @@
 import org.apache.hyracks.algebricks.runtime.evaluators.TupleFieldEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.operators.aggreg.AggregateRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.aggreg.NestedPlansAccumulatingAggregatorFactory;
-import org.apache.hyracks.algebricks.runtime.operators.aggreg.SimpleAlgebricksAccumulatingAggregatorFactory;
 import org.apache.hyracks.algebricks.runtime.operators.group.MicroPreClusteredGroupRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
 import org.apache.hyracks.algebricks.runtime.operators.meta.SubplanRuntimeFactory;
@@ -67,15 +66,12 @@
 import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import org.apache.hyracks.api.dataflow.value.INullWriterFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import org.apache.hyracks.data.std.primitive.IntegerPointable;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
@@ -85,7 +81,6 @@
 import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -93,8 +88,6 @@
 import org.apache.hyracks.dataflow.std.file.FileSplit;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.dataflow.std.file.LineFileWriteOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
-import org.apache.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.misc.SplitOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
@@ -483,75 +476,6 @@
     }
 
     @Test
-    public void scanHashGbySelectWrite() throws Exception {
-        JobSpecification spec = new JobSpecification(FRAME_SIZE);
-
-        // the scanner
-        FileSplit[] fileSplits = new FileSplit[1];
-        fileSplits[0] = new FileSplit(AlgebricksHyracksIntegrationUtil.NC1_ID, new FileReference(new File(
-                "data/tpch0.001/customer.tbl")));
-        IFileSplitProvider splitProvider = new ConstantFileSplitProvider(fileSplits);
-        RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE,
-                new UTF8StringSerializerDeserializer(), FloatSerializerDeserializer.INSTANCE,
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() });
-        IValueParserFactory[] valueParsers = new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
-                UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
-                UTF8StringParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                UTF8StringParserFactory.INSTANCE };
-        FileScanOperatorDescriptor scanner = new FileScanOperatorDescriptor(spec, splitProvider,
-                new DelimitedDataTupleParserFactory(valueParsers, '|'), scannerDesc);
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, scanner,
-                new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
-
-        // the group-by
-        RecordDescriptor gbyDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-        ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(new int[] { 3 },
-                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) });
-        IAggregateEvaluatorFactory[] aggFuns = new IAggregateEvaluatorFactory[] { new TupleCountAggregateFunctionFactory() };
-        IAggregatorDescriptorFactory aggFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(aggFuns,
-                new int[] { 3 });
-        HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(spec, new int[] { 3 }, tpcf,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                aggFactory, gbyDesc, 1024);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, gby,
-                new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
-
-        // the algebricks op.
-        IScalarEvaluatorFactory cond = new IntegerEqualsEvalFactory(new IntegerConstantEvalFactory(3),
-                new TupleFieldEvaluatorFactory(0)); // Canadian customers
-        StreamSelectRuntimeFactory select = new StreamSelectRuntimeFactory(cond, new int[] { 1 },
-                BinaryBooleanInspectorImpl.FACTORY, false, -1, null);
-        RecordDescriptor selectDesc = new RecordDescriptor(
-                new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE });
-
-        String filePath = PATH_ACTUAL + SEPARATOR + "scanHashGbySelectWrite.out";
-        File outFile = new File(filePath);
-        SinkWriterRuntimeFactory writer = new SinkWriterRuntimeFactory(new int[] { 0 },
-                new IPrinterFactory[] { IntegerPrinterFactory.INSTANCE }, outFile, PrinterBasedWriterFactory.INSTANCE,
-                selectDesc);
-
-        AlgebricksMetaOperatorDescriptor algebricksOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
-                new IPushRuntimeFactory[] { select, writer }, new RecordDescriptor[] { selectDesc, null });
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, algebricksOp,
-                new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), scanner, 0, gby, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), gby, 0, algebricksOp, 0);
-        spec.addRoot(algebricksOp);
-
-        AlgebricksHyracksIntegrationUtil.runJob(spec);
-        StringBuilder buf = new StringBuilder();
-        readFileToString(outFile, buf);
-        Assert.assertEquals("9", buf.toString());
-        outFile.delete();
-    }
-
-    @Test
     public void etsUnnestRunningaggregateWrite() throws Exception {
         JobSpecification spec = new JobSpecification(FRAME_SIZE);
 
diff --git a/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/FixedSizeFrame.java b/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/FixedSizeFrame.java
index 0afe0ae..e36a773 100644
--- a/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/FixedSizeFrame.java
+++ b/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/FixedSizeFrame.java
@@ -23,11 +23,19 @@
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class FixedSizeFrame implements IFrame{
+public class FixedSizeFrame implements IFrame {
 
-    private final ByteBuffer buffer;
+    private ByteBuffer buffer;
 
-    public FixedSizeFrame(ByteBuffer buffer){
+    public FixedSizeFrame() {
+
+    }
+
+    public FixedSizeFrame(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    public void reset(ByteBuffer buffer) {
         this.buffer = buffer;
     }
 
diff --git a/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameAppender.java b/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameAppender.java
index e48e4d6..0fc24c7 100644
--- a/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameAppender.java
+++ b/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/comm/IFrameAppender.java
@@ -24,6 +24,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IFrameAppender {
+
     /**
      * Reset to attach to a new frame.
      *
diff --git a/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksFrameMgrContext.java b/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksFrameMgrContext.java
index c945f8e..d43680a 100644
--- a/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksFrameMgrContext.java
+++ b/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksFrameMgrContext.java
@@ -24,9 +24,9 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IHyracksFrameMgrContext {
+
     int getInitialFrameSize();
 
-    //TODO tobedeleted
     ByteBuffer allocateFrame() throws HyracksDataException;
 
     ByteBuffer allocateFrame(int bytes) throws HyracksDataException;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FixedSizeFrameTupleAppender.java
similarity index 65%
copy from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
copy to hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FixedSizeFrameTupleAppender.java
index 21ed3e2..60c6e6d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FixedSizeFrameTupleAppender.java
@@ -17,10 +17,16 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.common.comm.io;
 
-public enum EnumFreeSlotPolicy {
-    SMALLEST_FIT,
-    LAST_FIT,
-    BIGGEST_FIT,
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class FixedSizeFrameTupleAppender extends FrameTupleAppender {
+    @Override
+    protected boolean canHoldNewTuple(int fieldCount, int dataLength) throws HyracksDataException {
+        if (hasEnoughSpace(fieldCount, dataLength)) {
+            return true;
+        }
+        return false;
+    }
 }
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
index 70d4f80..bab5463 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
@@ -135,13 +135,18 @@
     protected void prettyPrint(int tid, ByteBufferInputStream bbis, DataInputStream dis, StringBuilder sb) {
         sb.append(" tid" + tid + ":(" + getTupleStartOffset(tid) + ", " + getTupleEndOffset(tid) + ")[");
         for (int j = 0; j < getFieldCount(); ++j) {
+            sb.append(" ");
+            if (j > 0) {
+                sb.append("|");
+            }
             sb.append("f" + j + ":(" + getFieldStartOffset(tid, j) + ", " + getFieldEndOffset(tid, j) + ") ");
             sb.append("{");
             bbis.setByteBuffer(buffer, getTupleStartOffset(tid) + getFieldSlotsLength() + getFieldStartOffset(tid, j));
             try {
                 sb.append(recordDescriptor.getFields()[j].deserialize(dis));
-            } catch (HyracksDataException e) {
+            } catch (Exception e) {
                 e.printStackTrace();
+                sb.append("Failed to deserialize field" + j);
             }
             sb.append("}");
         }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/RunAndMaxFrameSizePair.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/GeneratedRunFileReader.java
similarity index 62%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/RunAndMaxFrameSizePair.java
rename to hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/GeneratedRunFileReader.java
index 1dfccfd..0967448 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/RunAndMaxFrameSizePair.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/GeneratedRunFileReader.java
@@ -17,20 +17,25 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort;
+package org.apache.hyracks.dataflow.common.io;
 
-import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IIOManager;
 
-public class RunAndMaxFrameSizePair {
-    public IFrameReader run;
-    public int maxFrameSize;
+public class GeneratedRunFileReader extends RunFileReader {
+    private int maxFrameSize;
 
-    public RunAndMaxFrameSizePair(IFrameReader run, int maxFrameSize) {
-        this.run = run;
+    GeneratedRunFileReader(FileReference file, IIOManager ioManager, long size, boolean deleteAfterRead,
+            int maxFrameSize) {
+        super(file, ioManager, size, deleteAfterRead);
         this.maxFrameSize = maxFrameSize;
     }
 
-    void updateSize(int newMaxSize){
+    public void updateSize(int newMaxSize) {
         this.maxFrameSize = newMaxSize;
     }
+
+    public int getMaxFrameSize() {
+        return maxFrameSize;
+    }
 }
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java
index b192be4..fb160f0 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java
@@ -54,6 +54,7 @@
             return false;
         }
         frame.reset();
+
         int readLength = ioManager.syncRead(handle, readPtr, frame.getBuffer());
         if (readLength <= 0) {
             throw new HyracksDataException("Premature end of file");
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
index a5888c1..8031422 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
@@ -33,6 +33,7 @@
 
     private IFileHandle handle;
     private long size;
+    private int maxOutputFrameSize;
 
     public RunFileWriter(FileReference file, IIOManager ioManager) {
         this.file = file;
@@ -45,6 +46,7 @@
                 IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
         size = 0;
         failed = false;
+        maxOutputFrameSize = 0;
     }
 
     @Override
@@ -55,7 +57,9 @@
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        size += ioManager.syncWrite(handle, size, buffer);
+        int writen = ioManager.syncWrite(handle, size, buffer);
+        maxOutputFrameSize = Math.max(writen, maxOutputFrameSize);
+        size += writen;
     }
 
     @Override
@@ -73,18 +77,18 @@
         return size;
     }
 
-    public RunFileReader createReader() throws HyracksDataException {
+    public GeneratedRunFileReader createReader() throws HyracksDataException {
         if (failed) {
             throw new HyracksDataException("createReader() called on a failed RunFileWriter");
         }
-        return new RunFileReader(file, ioManager, size, false);
+        return new GeneratedRunFileReader(file, ioManager, size, false, maxOutputFrameSize);
     }
 
-    public RunFileReader createDeleteOnCloseReader() throws HyracksDataException {
+    public GeneratedRunFileReader createDeleteOnCloseReader() throws HyracksDataException {
         if (failed) {
             throw new HyracksDataException("createReader() called on a failed RunFileWriter");
         }
-        return new RunFileReader(file, ioManager, size, true);
+        return new GeneratedRunFileReader(file, ioManager, size, true, maxOutputFrameSize);
     }
 
     @Override
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
index fb81912..fb9c6fb 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
@@ -44,7 +44,7 @@
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
 import org.apache.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortRunMerger;
-import org.apache.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
+import org.apache.hyracks.dataflow.std.structures.RunAndMaxFrameSizePair;
 
 public class ShuffleFrameReader implements IFrameReader {
     private final IHyracksTaskContext ctx;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTuplePointerAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTuplePointerAccessor.java
new file mode 100644
index 0000000..3683354
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTuplePointerAccessor.java
@@ -0,0 +1,134 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public abstract class AbstractTuplePointerAccessor implements ITuplePointerAccessor {
+
+    protected int tid = -1;
+
+    abstract IFrameTupleAccessor getInnerAccessor();
+
+    abstract void resetInnerAccessor(TuplePointer tuplePointer);
+
+    @Override
+    public void reset(TuplePointer tuplePointer) {
+        resetInnerAccessor(tuplePointer);
+        tid = tuplePointer.tupleIndex;
+    }
+
+    @Override
+    public int getTupleStartOffset() {
+        return getTupleStartOffset(tid);
+    }
+
+    @Override
+    public int getTupleLength() {
+        return getTupleLength(tid);
+    }
+
+    @Override
+    public int getAbsFieldStartOffset(int fieldId) {
+        return getAbsoluteFieldStartOffset(tid, fieldId);
+    }
+
+    @Override
+    public int getFieldLength(int fieldId) {
+        return getFieldLength(tid, fieldId);
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return getInnerAccessor().getBuffer();
+    }
+
+    @Override
+    public int getFieldCount() {
+        return getInnerAccessor().getFieldCount();
+    }
+
+    @Override
+    public int getFieldSlotsLength() {
+        return getInnerAccessor().getFieldSlotsLength();
+    }
+
+    protected void checkTupleIndex(int tupleIndex) {
+        if (tupleIndex != tid) {
+            throw new IllegalArgumentException(
+                    "ITupleBufferAccessor can not access the different tid other than the one given in reset function");
+        }
+    }
+
+    @Override
+    public int getFieldEndOffset(int tupleIndex, int fIdx) {
+        checkTupleIndex(tupleIndex);
+        return getInnerAccessor().getFieldEndOffset(tid, fIdx);
+    }
+
+    @Override
+    public int getFieldStartOffset(int tupleIndex, int fIdx) {
+        checkTupleIndex(tupleIndex);
+        return getInnerAccessor().getFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
+    public int getFieldLength(int tupleIndex, int fIdx) {
+        checkTupleIndex(tupleIndex);
+        return getInnerAccessor().getFieldLength(tupleIndex, fIdx);
+    }
+
+    @Override
+    public int getTupleLength(int tupleIndex) {
+        checkTupleIndex(tupleIndex);
+        return getInnerAccessor().getTupleLength(tupleIndex);
+    }
+
+    @Override
+    public int getTupleEndOffset(int tupleIndex) {
+        checkTupleIndex(tupleIndex);
+        return getInnerAccessor().getTupleEndOffset(tupleIndex);
+    }
+
+    @Override
+    public int getTupleStartOffset(int tupleIndex) {
+        checkTupleIndex(tupleIndex);
+        return getInnerAccessor().getTupleStartOffset(tupleIndex);
+    }
+
+    @Override
+    public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+        checkTupleIndex(tupleIndex);
+        return getInnerAccessor().getAbsoluteFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
+    public int getTupleCount() {
+        return getInnerAccessor().getTupleCount();
+    }
+
+    @Override
+    public void reset(ByteBuffer buffer) {
+        throw new IllegalAccessError("Should never call this reset");
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/BufferInfo.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/BufferInfo.java
new file mode 100644
index 0000000..d5fbdc0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/BufferInfo.java
@@ -0,0 +1,59 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.dataflow.std.structures.IResetable;
+
+public class BufferInfo implements IResetable<BufferInfo> {
+    private ByteBuffer buffer;
+    private int startOffset;
+    private int length;
+
+    public BufferInfo(ByteBuffer buffer, int startOffset, int length) {
+        reset(buffer, startOffset, length);
+    }
+
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    public int getStartOffset() {
+        return startOffset;
+    }
+
+    public int getLength() {
+        return length;
+    }
+
+    @Override
+    public void reset(BufferInfo other) {
+        this.buffer = other.buffer;
+        this.startOffset = other.startOffset;
+        this.length = other.length;
+    }
+
+    public void reset(ByteBuffer buffer, int startOffset, int length) {
+        this.buffer = buffer;
+        this.startOffset = startOffset;
+        this.length = length;
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/DeallocatableFramePool.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/DeallocatableFramePool.java
new file mode 100644
index 0000000..626edba
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/DeallocatableFramePool.java
@@ -0,0 +1,123 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.LinkedList;
+
+import org.apache.hyracks.api.context.IHyracksFrameMgrContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class DeallocatableFramePool implements IDeallocatableFramePool {
+
+    private final IHyracksFrameMgrContext ctx;
+    private final int memBudget;
+    private int allocated;
+    private LinkedList<ByteBuffer> buffers;
+
+    public DeallocatableFramePool(IHyracksFrameMgrContext ctx, int memBudgetInBytes) {
+        this.ctx = ctx;
+        this.memBudget = memBudgetInBytes;
+        this.allocated = 0;
+        this.buffers = new LinkedList<>();
+    }
+
+    @Override
+    public int getMinFrameSize() {
+        return ctx.getInitialFrameSize();
+    }
+
+    @Override
+    public int getMemoryBudgetBytes() {
+        return memBudget;
+    }
+
+    @Override
+    public ByteBuffer allocateFrame(int frameSize) throws HyracksDataException {
+        ByteBuffer buffer = findExistingFrame(frameSize);
+        if (buffer != null) {
+            return buffer;
+        }
+        if (haveEnoughFreeSpace(frameSize)) {
+            return createNewFrame(frameSize);
+        }
+        return mergeExistingFrames(frameSize);
+    }
+
+    private ByteBuffer mergeExistingFrames(int frameSize) throws HyracksDataException {
+        int mergedSize = memBudget - allocated;
+        for (Iterator<ByteBuffer> iter = buffers.iterator(); iter.hasNext(); ) {
+            ByteBuffer buffer = iter.next();
+            iter.remove();
+            mergedSize += buffer.capacity();
+            ctx.deallocateFrames(buffer.capacity());
+            allocated -= buffer.capacity();
+            if (mergedSize >= frameSize) {
+                return createNewFrame(mergedSize);
+            }
+        }
+        return null;
+
+    }
+
+    private ByteBuffer createNewFrame(int frameSize) throws HyracksDataException {
+        allocated += frameSize;
+        return ctx.allocateFrame(frameSize);
+    }
+
+    private boolean haveEnoughFreeSpace(int frameSize) {
+        return allocated + frameSize <= memBudget;
+    }
+
+    private ByteBuffer findExistingFrame(int frameSize) {
+        for (Iterator<ByteBuffer> iter = buffers.iterator(); iter.hasNext(); ) {
+            ByteBuffer next = iter.next();
+            if (next.capacity() >= frameSize) {
+                iter.remove();
+                return next;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public void deAllocateBuffer(ByteBuffer buffer) {
+        if (buffer.capacity() != ctx.getInitialFrameSize()) {
+            // simply deallocate the Big Object frame
+            ctx.deallocateFrames(buffer.capacity());
+            allocated -= buffer.capacity();
+        } else {
+            buffers.add(buffer);
+        }
+    }
+
+    @Override
+    public void reset() {
+        allocated = 0;
+        buffers.clear();
+    }
+
+    @Override
+    public void close() {
+        buffers.clear();
+        allocated = 0;
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/EnumFreeSlotPolicy.java
similarity index 68%
copy from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
copy to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/EnumFreeSlotPolicy.java
index 21ed3e2..15bbab2 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/EnumFreeSlotPolicy.java
@@ -17,10 +17,23 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
+/**
+ * Under the big-object setting, there could be multiple variable size free slots to use.
+ * In that case, we need to decide which free slot to give back to caller.
+ */
 public enum EnumFreeSlotPolicy {
+    /**
+     * Choose the minimum size frame
+     */
     SMALLEST_FIT,
+    /**
+     * Choose the latest used frame if it is big enough
+     */
     LAST_FIT,
+    /**
+     * Choose the largest size frame
+     */
     BIGGEST_FIT,
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirst.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
similarity index 85%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirst.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
index 200f27d..6f5587f 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirst.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
@@ -17,25 +17,26 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import org.apache.hyracks.dataflow.std.structures.IResetableComparable;
 import org.apache.hyracks.dataflow.std.structures.IResetableComparableFactory;
 import org.apache.hyracks.dataflow.std.structures.MaxHeap;
 
-public class FrameFreeSlotBiggestFirst implements IFrameFreeSlotPolicy {
+class FrameFreeSlotBiggestFirst implements IFrameFreeSlotPolicy {
     private static final int INVALID = -1;
+    private static final int INITIAL_FRAME_NUM = 10;
 
-    class SpaceEntryFactory implements IResetableComparableFactory {
+    protected class SpaceEntryFactory implements IResetableComparableFactory {
         @Override
         public IResetableComparable createResetableComparable() {
             return new SpaceEntry();
         }
     }
 
-    class SpaceEntry implements IResetableComparable<SpaceEntry> {
-        int space;
-        int id;
+    protected class SpaceEntry implements IResetableComparable<SpaceEntry> {
+        private int space;
+        private int id;
 
         SpaceEntry() {
             space = INVALID;
@@ -76,6 +77,10 @@
         tempEntry = new SpaceEntry();
     }
 
+    public FrameFreeSlotBiggestFirst() {
+        this(INITIAL_FRAME_NUM);
+    }
+
     @Override
     public int popBestFit(int tobeInsertedSize) {
         if (!heap.isEmpty()) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFit.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
similarity index 88%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFit.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
index 0235c6f..819ff80 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFit.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
@@ -17,16 +17,16 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import java.util.Arrays;
 
-public class FrameFreeSlotLastFit implements IFrameFreeSlotPolicy {
+class FrameFreeSlotLastFit implements IFrameFreeSlotPolicy {
     private static int INITIAL_CAPACITY = 10;
 
     private class FrameSpace {
-        int frameId;
-        int freeSpace;
+        private int frameId;
+        private int freeSpace;
 
         FrameSpace(int frameId, int freeSpace) {
             reset(frameId, freeSpace);
@@ -41,8 +41,8 @@
     private FrameSpace[] frameSpaces;
     private int size;
 
-    public FrameFreeSlotLastFit(int maxFrames) {
-        frameSpaces = new FrameSpace[maxFrames];
+    public FrameFreeSlotLastFit(int initialFrameNumber) {
+        frameSpaces = new FrameSpace[initialFrameNumber];
         size = 0;
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotPolicyFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotPolicyFactory.java
new file mode 100644
index 0000000..d4267ab
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotPolicyFactory.java
@@ -0,0 +1,46 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+public class FrameFreeSlotPolicyFactory {
+    public static IFrameFreeSlotPolicy createFreeSlotPolicy(EnumFreeSlotPolicy policy, int initialNumberFrames) {
+        switch (policy) {
+            case SMALLEST_FIT:
+                return new FrameFreeSlotSmallestFit();
+            case BIGGEST_FIT:
+                return new FrameFreeSlotBiggestFirst(initialNumberFrames);
+            case LAST_FIT:
+            default:
+                return new FrameFreeSlotLastFit(initialNumberFrames);
+        }
+    }
+
+    public static IFrameFreeSlotPolicy createFreeSlotPolicy(EnumFreeSlotPolicy policy) {
+        switch (policy) {
+            case SMALLEST_FIT:
+                return new FrameFreeSlotSmallestFit();
+            case BIGGEST_FIT:
+                return new FrameFreeSlotBiggestFirst();
+            case LAST_FIT:
+            default:
+                return new FrameFreeSlotLastFit();
+        }
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotSmallestFit.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
similarity index 93%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotSmallestFit.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
index 8331a93..ada6752 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotSmallestFit.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
@@ -17,13 +17,13 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.TreeMap;
 
-public class FrameFreeSlotSmallestFit implements IFrameFreeSlotPolicy {
+class FrameFreeSlotSmallestFit implements IFrameFreeSlotPolicy {
 
     private TreeMap<Integer, LinkedList<Integer>> freeSpaceIndex;
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IDeallocatableFramePool.java
similarity index 80%
copy from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
copy to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IDeallocatableFramePool.java
index 21ed3e2..39426c1 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IDeallocatableFramePool.java
@@ -17,10 +17,12 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
-public enum EnumFreeSlotPolicy {
-    SMALLEST_FIT,
-    LAST_FIT,
-    BIGGEST_FIT,
+import java.nio.ByteBuffer;
+
+public interface IDeallocatableFramePool extends IFramePool {
+
+    void deAllocateBuffer(ByteBuffer buffer);
+
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IDeletableTupleBufferManager.java
similarity index 71%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IDeletableTupleBufferManager.java
index 00decb9..71d08b2 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IDeletableTupleBufferManager.java
@@ -17,24 +17,12 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
-import java.nio.ByteBuffer;
-
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.std.structures.TuplePointer;
 
-public interface ITupleBufferAccessor {
+public interface IDeletableTupleBufferManager extends ITupleBufferManager {
 
-    void reset(TuplePointer tuplePointer);
-
-    ByteBuffer getTupleBuffer();
-
-    int getTupleStartOffset();
-
-    int getTupleLength();
-
-    int getAbsFieldStartOffset(int fieldId);
-
-    int getFieldLength(int fieldId);
-
+    void deleteTuple(TuplePointer tuplePointer) throws HyracksDataException;
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameBufferManager.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
similarity index 67%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameBufferManager.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
index 1cf454d..1118bf3 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameBufferManager.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
@@ -16,12 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import java.nio.ByteBuffer;
 
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
+/**
+ * Manage the buffer space in the unit of frame.
+ */
 public interface IFrameBufferManager {
 
     /**
@@ -33,26 +36,10 @@
 
     /**
      * @param frameIndex
-     * @return the specified frame, from the set of memory buffers, being
-     * managed by this memory manager
+     * @param bufferInfo the given object need to be reset
+     * @return the filled bufferInfo to facilitate the chain access
      */
-    ByteBuffer getFrame(int frameIndex);
-
-    /**
-     * Get the startOffset of the specific frame inside buffer
-     *
-     * @param frameIndex
-     * @return the start offset of the frame returned by {@link #getFrame(int)} method.
-     */
-    int getFrameStartOffset(int frameIndex);
-
-    /**
-     * Get the size of the specific frame inside buffer
-     *
-     * @param frameIndex
-     * @return the length of the specific frame
-     */
-    int getFrameSize(int frameIndex);
+    BufferInfo getFrame(int frameIndex, BufferInfo bufferInfo);
 
     /**
      * @return the number of frames in this buffer
@@ -63,7 +50,7 @@
      * Writes the whole frame into the buffer.
      *
      * @param frame source frame
-     * @return the id of the inserted frame. if failed to return it will be -1.
+     * @return the id of the inserted frame. return -1 if it failed to insert
      */
     int insertFrame(ByteBuffer frame) throws HyracksDataException;
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameFreeSlotPolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
similarity index 96%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameFreeSlotPolicy.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
index fab1706..8a1e004 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameFreeSlotPolicy.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 public interface IFrameFreeSlotPolicy {
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFramePool.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFramePool.java
similarity index 96%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFramePool.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFramePool.java
index f555971..2c6b01e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFramePool.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFramePool.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import java.nio.ByteBuffer;
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedMemoryConstrain.java
similarity index 69%
copy from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
copy to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedMemoryConstrain.java
index 21ed3e2..1ba0745 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedMemoryConstrain.java
@@ -17,10 +17,18 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
-public enum EnumFreeSlotPolicy {
-    SMALLEST_FIT,
-    LAST_FIT,
-    BIGGEST_FIT,
+/**
+ * Provide a customized constraint for different partitions.
+ */
+public interface IPartitionedMemoryConstrain {
+    /**
+     * Set number of frames that can be used by the specific partition.
+     * 
+     * @param partitionId
+     * @return
+     */
+    int frameLimit(int partitionId);
+
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
new file mode 100644
index 0000000..70ca2dd
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
@@ -0,0 +1,111 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public interface IPartitionedTupleBufferManager {
+
+    int getNumPartitions();
+
+    int getNumTuples(int partition);
+
+    int getPhysicalSize(int partition);
+
+    /**
+     * Insert tuple from (byte[] byteArray,int[] fieldEndOffsets, int start, int size) into
+     * specified partition. The handle is written into the tuplepointer.
+     * <br>
+     * If {@code byteArray} contains the {@code fieldEndOffsets} already, then please set the {@code fieldEndOffsets} as NULL
+     *
+     * @param partition
+     *            the id of the partition to insert the tuple into
+     * @param byteArray
+     *            the byteArray which contains the tuple
+     * @param fieldEndOffsets
+     *            the fieldEndOffsets which comes from the ArrayTupleBuilder, please set it to NULL if the {@code byteArray} already contains the fieldEndOffsets
+     * @param start
+     *            the start offset in the {@code byteArray}
+     * @param size
+     *            the size of the tuple
+     * @param pointer
+     *            the returned pointer indicating the handler inside this buffer manager
+     * @return a boolean value to indicate if the insertion succeed or not
+     */
+    boolean insertTuple(int partition, byte[] byteArray, int[] fieldEndOffsets, int start, int size,
+            TuplePointer pointer) throws HyracksDataException;
+
+    /**
+     * Insert tuple {@code tupleId} from the {@code tupleAccessor} into the given partition.
+     * The returned handle is written into the tuplepointer
+     * 
+     * @param partition
+     *            the id of the partition to insert the tuple
+     * @param tupleAccessor
+     *            the FrameTupleAccessor storage
+     * @param tupleId
+     *            the id of the tuple from the tupleAccessor
+     * @param pointer
+     *            the returned pointer indicating the handler to later fetch the tuple from the buffer maanager
+     * @return true if the insertion succeed. Otherwise return false.
+     * @throws HyracksDataException
+     */
+    boolean insertTuple(int partition, IFrameTupleAccessor tupleAccessor, int tupleId, TuplePointer pointer)
+            throws HyracksDataException;
+
+    /**
+     * Reset to the initial states. The previous allocated resources won't be released in order to be used in the next round.
+     * 
+     * @throws HyracksDataException
+     */
+    void reset() throws HyracksDataException;
+
+    /**
+     * Close the managers which will explicitly release all the allocated resources.
+     */
+    void close();
+
+    ITuplePointerAccessor getTupleAccessor(RecordDescriptor recordDescriptor);
+
+    /**
+     * Flush the particular partition {@code pid} to {@code writer}.
+     * This partition will not be cleared.
+     * Currently it is used by Join where we flush the inner partition to the join (as a frameWriter),
+     * but we will still keep the inner for the next outer partition.
+     * 
+     * @param pid
+     * @param writer
+     * @throws HyracksDataException
+     */
+    void flushPartition(int pid, IFrameWriter writer) throws HyracksDataException;
+
+    /**
+     * Clear the memory occupation of the particular partition.
+     * 
+     * @param partition
+     * @throws HyracksDataException
+     */
+    void clearPartition(int partition) throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
similarity index 84%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
index ae502a0..324401a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
@@ -17,12 +17,15 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.std.structures.TuplePointer;
 
+/**
+ * Manage the buffer space. Different from the {@link IFrameBufferManager}, this one allows the record level manipulation.
+ */
 public interface ITupleBufferManager {
     /**
      * Reset the counters and flags to initial status. This method should not release the pre-allocated resources
@@ -38,9 +41,7 @@
 
     boolean insertTuple(IFrameTupleAccessor accessor, int idx, TuplePointer tuplePointer) throws HyracksDataException;
 
-    void deleteTuple(TuplePointer tuplePointer) throws HyracksDataException;
+    void close() throws HyracksDataException;
 
-    void close();
-
-    ITupleBufferAccessor getTupleAccessor();
+    ITuplePointerAccessor createTupleAccessor();
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITuplePointerAccessor.java
similarity index 62%
copy from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java
copy to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITuplePointerAccessor.java
index 00decb9..a8ab7eb 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITuplePointerAccessor.java
@@ -17,18 +17,20 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
-import java.nio.ByteBuffer;
-
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.dataflow.std.structures.TuplePointer;
 
-public interface ITupleBufferAccessor {
-
+/**
+ * A cursor-like tuple level accessor to point to a tuple physical byte location inside the {@link ITupleBufferManager}
+ * Some of the BufferManger (e.g. {@link VariableDeletableTupleMemoryManager}are using the different frame structure as
+ * the common {@link org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor} does.
+ * In order to hide the complexity inside the buffer manager, clients can use this Accessor to navigate the internal record.
+ */
+public interface ITuplePointerAccessor extends IFrameTupleAccessor {
     void reset(TuplePointer tuplePointer);
 
-    ByteBuffer getTupleBuffer();
-
     int getTupleStartOffset();
 
     int getTupleLength();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java
new file mode 100644
index 0000000..f9387a9
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java
@@ -0,0 +1,99 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import java.util.BitSet;
+import java.util.function.IntUnaryOperator;
+
+/**
+ * This policy is used to decide which partition in {@link VPartitionTupleBufferManager} should be a victim when
+ * there is not enough space to insert new element.
+ */
+public class PreferToSpillFullyOccupiedFramePolicy {
+
+    private final IPartitionedTupleBufferManager bufferManager;
+    private final BitSet spilledStatus;
+    private final int minFrameSize;
+
+    public PreferToSpillFullyOccupiedFramePolicy(IPartitionedTupleBufferManager bufferManager, BitSet spilledStatus,
+            int minFrameSize) {
+        this.bufferManager = bufferManager;
+        this.spilledStatus = spilledStatus;
+        this.minFrameSize = minFrameSize;
+    }
+
+    public int selectVictimPartition(int failedToInsertPartition) {
+        // To avoid flush the half-full frame, it's better to spill itself.
+        if (bufferManager.getNumTuples(failedToInsertPartition) > 0) {
+            return failedToInsertPartition;
+        }
+        int partitionToSpill = findSpilledPartitionWithMaxMemoryUsage();
+        int maxToSpillPartSize = 0;
+        // if we couldn't find the already spilled partition, or it is too small to flush that one,
+        // try to flush an in memory partition.
+        if (partitionToSpill < 0
+                || (maxToSpillPartSize = bufferManager.getPhysicalSize(partitionToSpill)) == minFrameSize) {
+            int partitionInMem = findInMemPartitionWithMaxMemoryUsage();
+            if (partitionInMem >= 0 && bufferManager.getPhysicalSize(partitionInMem) > maxToSpillPartSize) {
+                partitionToSpill = partitionInMem;
+            }
+        }
+        return partitionToSpill;
+    }
+
+    public int findInMemPartitionWithMaxMemoryUsage() {
+        return findMaxSize(spilledStatus.nextClearBit(0), (i) -> spilledStatus.nextClearBit(i + 1));
+    }
+
+    public int findSpilledPartitionWithMaxMemoryUsage() {
+        return findMaxSize(spilledStatus.nextSetBit(0), (i) -> spilledStatus.nextSetBit(i + 1));
+    }
+
+    private int findMaxSize(int startIndex, IntUnaryOperator nextIndexOp) {
+        int pid = -1;
+        int max = 0;
+        for (int i = startIndex; i >= 0 && i < bufferManager.getNumPartitions(); i = nextIndexOp.applyAsInt(i)) {
+            int partSize = bufferManager.getPhysicalSize(i);
+            if (partSize > max) {
+                max = partSize;
+                pid = i;
+            }
+        }
+        return pid;
+    }
+
+    /**
+     * Create an constrain for the already spilled partition that it can only use at most one frame.
+     * 
+     * @param spillStatus
+     * @return
+     */
+    public static IPartitionedMemoryConstrain createAtMostOneFrameForSpilledPartitionConstrain(BitSet spillStatus) {
+        return new IPartitionedMemoryConstrain() {
+            @Override
+            public int frameLimit(int partitionId) {
+                if (spillStatus.get(partitionId)) {
+                    return 1;
+                }
+                return Integer.MAX_VALUE;
+            }
+        };
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
new file mode 100644
index 0000000..ed54973
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
@@ -0,0 +1,277 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.hyracks.api.comm.FixedSizeFrame;
+import org.apache.hyracks.api.comm.FrameHelper;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksFrameMgrContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FixedSizeFrameTupleAppender;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+/**
+ * This buffer manager will dived the buffers into given number of partitions.
+ * The cleared partition (spilled one in the caller side) can only get no more than one frame.
+ */
+public class VPartitionTupleBufferManager implements IPartitionedTupleBufferManager {
+
+    private IDeallocatableFramePool framePool;
+    private IFrameBufferManager[] partitionArray;
+    private int[] numTuples;
+    private final FixedSizeFrame appendFrame;
+    private final FixedSizeFrameTupleAppender appender;
+    private BufferInfo tempInfo;
+    private final IPartitionedMemoryConstrain constrain;
+
+    public VPartitionTupleBufferManager(IHyracksFrameMgrContext ctx, IPartitionedMemoryConstrain constrain,
+            int partitions, int frameLimitInBytes) throws HyracksDataException {
+        this.constrain = constrain;
+        this.framePool = new DeallocatableFramePool(ctx, frameLimitInBytes);
+        this.partitionArray = new IFrameBufferManager[partitions];
+        this.numTuples = new int[partitions];
+        this.appendFrame = new FixedSizeFrame();
+        this.appender = new FixedSizeFrameTupleAppender();
+        this.tempInfo = new BufferInfo(null, -1, -1);
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        for (IFrameBufferManager part : partitionArray) {
+            if (part != null) {
+                for (int i = 0; i < part.getNumFrames(); i++) {
+                    framePool.deAllocateBuffer(part.getFrame(i, tempInfo).getBuffer());
+                }
+                part.reset();
+            }
+        }
+        Arrays.fill(numTuples, 0);
+        appendFrame.reset(null);
+    }
+
+    @Override
+    public int getNumPartitions() {
+        return partitionArray.length;
+    }
+
+    @Override
+    public int getNumTuples(int partition) {
+        return numTuples[partition];
+    }
+
+    @Override
+    public int getPhysicalSize(int partitionId) {
+        int size = 0;
+        IFrameBufferManager partition = partitionArray[partitionId];
+        if (partition != null) {
+            for (int i = 0; i < partition.getNumFrames(); ++i) {
+                size += partition.getFrame(i, tempInfo).getLength();
+            }
+        }
+        return size;
+    }
+
+    @Override
+    public void clearPartition(int partitionId) throws HyracksDataException {
+        IFrameBufferManager partition = partitionArray[partitionId];
+        if (partition != null) {
+            for (int i = 0; i < partition.getNumFrames(); ++i) {
+                framePool.deAllocateBuffer(partition.getFrame(i, tempInfo).getBuffer());
+            }
+        }
+        partitionArray[partitionId].reset();
+        numTuples[partitionId] = 0;
+    }
+
+    @Override
+    public boolean insertTuple(int partition, byte[] byteArray, int[] fieldEndOffsets, int start, int size,
+            TuplePointer pointer) throws HyracksDataException {
+        int actualSize = calculateActualSize(fieldEndOffsets, size);
+        int fid = getLastBufferOrCreateNewIfNotExist(partition, actualSize);
+        if (fid < 0) {
+            return false;
+        }
+        partitionArray[partition].getFrame(fid, tempInfo);
+        int tid = appendTupleToBuffer(tempInfo, fieldEndOffsets, byteArray, start, size);
+        if (tid < 0) {
+            if (partitionArray[partition].getNumFrames() >= constrain.frameLimit(partition)) {
+                return false;
+            }
+            fid = createNewBuffer(partition, actualSize);
+            if (fid < 0) {
+                return false;
+            }
+            partitionArray[partition].getFrame(fid, tempInfo);
+            tid = appendTupleToBuffer(tempInfo, fieldEndOffsets, byteArray, start, size);
+        }
+        pointer.reset(makeGroupFrameId(partition, fid), tid);
+        numTuples[partition]++;
+        return true;
+    }
+
+    @Override
+    public boolean insertTuple(int partition, IFrameTupleAccessor tupleAccessor, int tupleId, TuplePointer pointer)
+            throws HyracksDataException {
+        return insertTuple(partition, tupleAccessor.getBuffer().array(), null,
+                tupleAccessor.getTupleStartOffset(tupleId), tupleAccessor.getTupleLength(tupleId), pointer);
+    }
+
+    private static int calculateActualSize(int[] fieldEndOffsets, int size) {
+        if (fieldEndOffsets != null) {
+            return FrameHelper.calcRequiredSpace(fieldEndOffsets.length, size);
+        }
+        return FrameHelper.calcRequiredSpace(0, size);
+    }
+
+    private int makeGroupFrameId(int partition, int fid) {
+        return fid * getNumPartitions() + partition;
+    }
+
+    private int parsePartitionId(int externalFrameId) {
+        return externalFrameId % getNumPartitions();
+    }
+
+    private int parseFrameIdInPartition(int externalFrameId) {
+        return externalFrameId / getNumPartitions();
+    }
+
+    private int createNewBuffer(int partition, int size) throws HyracksDataException {
+        ByteBuffer newBuffer = requestNewBufferFromPool(size);
+        if (newBuffer == null) {
+            return -1;
+        }
+        appendFrame.reset(newBuffer);
+        appender.reset(appendFrame, true);
+        return partitionArray[partition].insertFrame(newBuffer);
+    }
+
+    private ByteBuffer requestNewBufferFromPool(int recordSize) throws HyracksDataException {
+        int frameSize = FrameHelper.calcAlignedFrameSizeToStore(0, recordSize, framePool.getMinFrameSize());
+        return framePool.allocateFrame(frameSize);
+    }
+
+    private int appendTupleToBuffer(BufferInfo bufferInfo, int[] fieldEndOffsets, byte[] byteArray, int start, int size)
+            throws HyracksDataException {
+        assert (bufferInfo.getStartOffset() == 0) : "Haven't supported yet in FrameTupleAppender";
+        if (bufferInfo.getBuffer() != appendFrame.getBuffer()) {
+            appendFrame.reset(bufferInfo.getBuffer());
+            appender.reset(appendFrame, false);
+        }
+        if (fieldEndOffsets == null) {
+            if (appender.append(byteArray, start, size)) {
+                return appender.getTupleCount() - 1;
+            }
+        } else {
+            if (appender.append(fieldEndOffsets, byteArray, start, size)) {
+                return appender.getTupleCount() - 1;
+            }
+        }
+
+        return -1;
+    }
+
+    private int getLastBufferOrCreateNewIfNotExist(int partition, int actualSize) throws HyracksDataException {
+        if (partitionArray[partition] == null || partitionArray[partition].getNumFrames() == 0) {
+            partitionArray[partition] = new PartitionFrameBufferManager();
+            return createNewBuffer(partition, actualSize);
+        }
+        return partitionArray[partition].getNumFrames() - 1;
+    }
+
+    @Override
+    public void close() {
+        framePool.close();
+        Arrays.fill(partitionArray, null);
+    }
+
+    private class PartitionFrameBufferManager implements IFrameBufferManager {
+
+        ArrayList<ByteBuffer> buffers = new ArrayList<>();
+
+        @Override
+        public void reset() throws HyracksDataException {
+            buffers.clear();
+        }
+
+        @Override
+        public BufferInfo getFrame(int frameIndex, BufferInfo returnedInfo) {
+            returnedInfo.reset(buffers.get(frameIndex), 0, buffers.get(frameIndex).capacity());
+            return returnedInfo;
+        }
+
+        @Override
+        public int getNumFrames() {
+            return buffers.size();
+        }
+
+        @Override
+        public int insertFrame(ByteBuffer frame) throws HyracksDataException {
+            buffers.add(frame);
+            return buffers.size() - 1;
+        }
+
+        @Override
+        public void close() {
+            buffers = null;
+        }
+
+    }
+
+    @Override
+    public ITuplePointerAccessor getTupleAccessor(final RecordDescriptor recordDescriptor) {
+        return new AbstractTuplePointerAccessor() {
+            FrameTupleAccessor innerAccessor = new FrameTupleAccessor(recordDescriptor);
+
+            @Override
+            IFrameTupleAccessor getInnerAccessor() {
+                return innerAccessor;
+            }
+
+            @Override
+            void resetInnerAccessor(TuplePointer tuplePointer) {
+                partitionArray[parsePartitionId(tuplePointer.frameIndex)]
+                        .getFrame(parseFrameIdInPartition(tuplePointer.frameIndex), tempInfo);
+                innerAccessor.reset(tempInfo.getBuffer(), tempInfo.getStartOffset(), tempInfo.getLength());
+            }
+        };
+    }
+
+    @Override
+    public void flushPartition(int pid, IFrameWriter writer) throws HyracksDataException {
+        IFrameBufferManager partition = partitionArray[pid];
+        if (partition != null && getNumTuples(pid) > 0) {
+            for (int i = 0; i < partition.getNumFrames(); ++i) {
+                partition.getFrame(i, tempInfo);
+                tempInfo.getBuffer().position(tempInfo.getStartOffset());
+                tempInfo.getBuffer().limit(tempInfo.getStartOffset() + tempInfo.getLength());
+                writer.nextFrame(tempInfo.getBuffer());
+            }
+        }
+
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
similarity index 82%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
index 20642bf..4359e49 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -31,9 +31,12 @@
 import org.apache.hyracks.dataflow.std.sort.util.IAppendDeletableFrameTupleAccessor;
 import org.apache.hyracks.dataflow.std.structures.TuplePointer;
 
-public class VariableTupleMemoryManager implements ITupleBufferManager {
+/**
+ * Enable the delete record operation in the memory management. This is only used in the {@link org.apache.hyracks.dataflow.std.sort.HeapSortRunGenerator}
+ */
+public class VariableDeletableTupleMemoryManager implements IDeletableTupleBufferManager {
 
-    private final static Logger LOG = Logger.getLogger(VariableTupleMemoryManager.class.getName());
+    private final static Logger LOG = Logger.getLogger(VariableDeletableTupleMemoryManager.class.getName());
 
     private final int MIN_FREE_SPACE;
     private final IFramePool pool;
@@ -44,7 +47,7 @@
     private int numTuples;
     private int statsReOrg;
 
-    public VariableTupleMemoryManager(IFramePool framePool, RecordDescriptor recordDescriptor) {
+    public VariableDeletableTupleMemoryManager(IFramePool framePool, RecordDescriptor recordDescriptor) {
         this.pool = framePool;
         int maxFrames = framePool.getMemoryBudgetBytes() / framePool.getMinFrameSize();
         this.policy = new FrameFreeSlotLastFit(maxFrames);
@@ -165,41 +168,19 @@
     }
 
     @Override
-    public ITupleBufferAccessor getTupleAccessor() {
-        return new ITupleBufferAccessor() {
+    public ITuplePointerAccessor createTupleAccessor() {
+        return new AbstractTuplePointerAccessor() {
             private IAppendDeletableFrameTupleAccessor bufferAccessor = new DeletableFrameTupleAppender(
                     recordDescriptor);
-            private int tid;
 
             @Override
-            public void reset(TuplePointer tuplePointer) {
+            IFrameTupleAccessor getInnerAccessor() {
+                return bufferAccessor;
+            }
+
+            @Override
+            void resetInnerAccessor(TuplePointer tuplePointer) {
                 bufferAccessor.reset(frames.get(tuplePointer.frameIndex));
-                tid = tuplePointer.tupleIndex;
-            }
-
-            @Override
-            public ByteBuffer getTupleBuffer() {
-                return bufferAccessor.getBuffer();
-            }
-
-            @Override
-            public int getTupleStartOffset() {
-                return bufferAccessor.getTupleStartOffset(tid);
-            }
-
-            @Override
-            public int getTupleLength() {
-                return bufferAccessor.getTupleLength(tid);
-            }
-
-            @Override
-            public int getAbsFieldStartOffset(int fieldId) {
-                return bufferAccessor.getAbsoluteFieldStartOffset(tid, fieldId);
-            }
-
-            @Override
-            public int getFieldLength(int fieldId) {
-                return bufferAccessor.getFieldLength(tid, fieldId);
             }
         };
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFrameMemoryManager.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
similarity index 64%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFrameMemoryManager.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
index 444b0b6..fe7de23 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFrameMemoryManager.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
@@ -17,53 +17,38 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.hyracks.api.comm.FixedSizeFrame;
 import org.apache.hyracks.api.comm.FrameHelper;
-import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.util.IntSerDeUtils;
 
 public class VariableFrameMemoryManager implements IFrameBufferManager {
 
-    private class PhysicalFrameOffset {
-        IFrame physicalFrame;
+    class PhysicalFrameOffset {
+        ByteBuffer physicalFrame;
         int physicalOffset;
 
-        PhysicalFrameOffset(IFrame frame, int offset) {
+        PhysicalFrameOffset(ByteBuffer frame, int offset) {
             physicalFrame = frame;
             physicalOffset = offset;
         }
     }
 
-    private class LogicalFrameStartSize {
-        ByteBuffer logicalFrame;
-        int logicalStart;
-        int logicalSize;
-
-        LogicalFrameStartSize(ByteBuffer frame, int start, int size) {
-            logicalFrame = frame;
-            logicalStart = start;
-            logicalSize = size;
-        }
-    }
-
     private final IFramePool framePool;
     private List<PhysicalFrameOffset> physicalFrameOffsets;
-    private List<LogicalFrameStartSize> logicalFrameStartSizes;
+    private List<BufferInfo> logicalFrameStartSizes;
     private final IFrameFreeSlotPolicy freeSlotPolicy;
 
     public VariableFrameMemoryManager(IFramePool framePool, IFrameFreeSlotPolicy freeSlotPolicy) {
         this.framePool = framePool;
         this.freeSlotPolicy = freeSlotPolicy;
-        int maxFrames = framePool.getMemoryBudgetBytes() / framePool.getMinFrameSize();
-        this.physicalFrameOffsets = new ArrayList<>(maxFrames);
-        this.logicalFrameStartSizes = new ArrayList<>(maxFrames);
+        this.physicalFrameOffsets = new ArrayList<>();
+        this.logicalFrameStartSizes = new ArrayList<>();
     }
 
     private int findAvailableFrame(int frameSize) throws HyracksDataException {
@@ -74,7 +59,7 @@
         ByteBuffer buffer = framePool.allocateFrame(frameSize);
         if (buffer != null) {
             IntSerDeUtils.putInt(buffer.array(), FrameHelper.getTupleCountOffset(buffer.capacity()), 0);
-            physicalFrameOffsets.add(new PhysicalFrameOffset(new FixedSizeFrame(buffer), 0));
+            physicalFrameOffsets.add(new PhysicalFrameOffset(buffer, 0));
             return physicalFrameOffsets.size() - 1;
         }
         return -1;
@@ -89,18 +74,9 @@
     }
 
     @Override
-    public ByteBuffer getFrame(int frameIndex) {
-        return logicalFrameStartSizes.get(frameIndex).logicalFrame;
-    }
-
-    @Override
-    public int getFrameStartOffset(int frameIndex) {
-        return logicalFrameStartSizes.get(frameIndex).logicalStart;
-    }
-
-    @Override
-    public int getFrameSize(int frameIndex) {
-        return logicalFrameStartSizes.get(frameIndex).logicalSize;
+    public BufferInfo getFrame(int frameIndex, BufferInfo info) {
+        info.reset(logicalFrameStartSizes.get(frameIndex));
+        return info;
     }
 
     @Override
@@ -115,14 +91,15 @@
         if (physicalFrameId < 0) {
             return -1;
         }
-        ByteBuffer buffer = physicalFrameOffsets.get(physicalFrameId).physicalFrame.getBuffer();
-        int offset = physicalFrameOffsets.get(physicalFrameId).physicalOffset;
+        PhysicalFrameOffset frameOffset = physicalFrameOffsets.get(physicalFrameId);
+        ByteBuffer buffer = frameOffset.physicalFrame;
+        int offset = frameOffset.physicalOffset;
         System.arraycopy(frame.array(), 0, buffer.array(), offset, frameSize);
         if (offset + frameSize < buffer.capacity()) {
             freeSlotPolicy.pushNewFrame(physicalFrameId, buffer.capacity() - offset - frameSize);
         }
-        physicalFrameOffsets.get(physicalFrameId).physicalOffset = offset + frameSize;
-        logicalFrameStartSizes.add(new LogicalFrameStartSize(buffer, offset, frameSize));
+        frameOffset.physicalOffset = offset + frameSize;
+        logicalFrameStartSizes.add(new BufferInfo(buffer, offset, frameSize));
         return logicalFrameStartSizes.size() - 1;
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFramePool.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFramePool.java
similarity index 88%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFramePool.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFramePool.java
index 344f961..f83eb8b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFramePool.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFramePool.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -37,14 +37,15 @@
     private final int memBudget;
 
     private int allocateMem;
-    private ArrayList<ByteBuffer> buffers;  // the unused slots were sorted by size increasingly.
+    private ArrayList<ByteBuffer> buffers; // the unused slots were sorted by size increasingly.
     private BitSet used; // the merged one also marked as used.
 
     /**
      * The constructor of the VariableFramePool.
      *
      * @param ctx
-     * @param memBudgetInBytes the given memory budgets to allocate the frames. If it less than 0, it will be treated as unlimited budgets
+     * @param memBudgetInBytes
+     *            the given memory budgets to allocate the frames. If it less than 0, it will be treated as unlimited budgets
      */
     public VariableFramePool(IHyracksFrameMgrContext ctx, int memBudgetInBytes) {
         this.ctx = ctx;
@@ -88,17 +89,17 @@
         return frameSize + allocateMem <= memBudget;
     }
 
-    private static int getFirstUnUsedPos(BitSet used) {
+    private static int getFirstUnusedPos(BitSet used) {
         return used.nextClearBit(0);
     }
 
-    private static int getLastUnUsedPos(BitSet used, int lastPos) {
+    private static int getLastUnusedPos(BitSet used, int lastPos) {
         return used.previousClearBit(lastPos);
     }
 
     private static int binarySearchUnusedBuffer(ArrayList<ByteBuffer> buffers, BitSet used, int frameSize) {
-        int l = getFirstUnUsedPos(used); // to skip the merged null buffers
-        int h = getLastUnUsedPos(used, (buffers.size() - 1)) + 1; // to skip the newly created buffers
+        int l = getFirstUnusedPos(used); // to skip the merged null buffers
+        int h = getLastUnusedPos(used, (buffers.size() - 1)) + 1; // to skip the newly created buffers
         if (l >= h) {
             return -1;
         }
@@ -147,8 +148,8 @@
      */
     private ByteBuffer mergeExistingFrames(int frameSize) throws HyracksDataException {
         int mergedSize = memBudget - allocateMem;
-        int highBound = getLastUnUsedPos(used, buffers.size() - 1) + 1;
-        for (int i = getFirstUnUsedPos(used); i < highBound; ++i) {
+        int highBound = getLastUnusedPos(used, buffers.size() - 1) + 1;
+        for (int i = getFirstUnusedPos(used); i < highBound; ++i) {
             if (!used.get(i)) {
                 mergedSize += deAllocateFrame(i);
                 if (mergedSize >= frameSize) {
@@ -176,7 +177,7 @@
     }
 
     private static void removeEmptySpot(List<ByteBuffer> buffers) {
-        for (int i = 0; i < buffers.size(); ) {
+        for (int i = 0; i < buffers.size();) {
             if (buffers.get(i) == null) {
                 buffers.remove(i);
             } else {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractRunningAggregatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractRunningAggregatorDescriptor.java
deleted file mode 100644
index 06c46b0..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AbstractRunningAggregatorDescriptor.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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 org.apache.hyracks.dataflow.std.group;
-
-import org.apache.hyracks.api.comm.IFrameTupleAccessor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-public abstract class AbstractRunningAggregatorDescriptor implements IAggregatorDescriptor {
-
-    /* (non-Javadoc)
-     * @see org.apache.hyracks.dataflow.std.group.IAggregatorDescriptor#outputPartialResult(org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder, org.apache.hyracks.api.comm.IFrameTupleAccessor, int, org.apache.hyracks.dataflow.std.group.AggregateState)
-     */
-    @Override
-    public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
-            AggregateState state) throws HyracksDataException {
-        return false;
-    }
-
-    /* (non-Javadoc)
-     * @see org.apache.hyracks.dataflow.std.group.IAggregatorDescriptor#outputFinalResult(org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder, org.apache.hyracks.api.comm.IFrameTupleAccessor, int, org.apache.hyracks.dataflow.std.group.AggregateState)
-     */
-    @Override
-    public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
-            AggregateState state) throws HyracksDataException {
-        return false;
-    }
-
-}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AggregateType.java
similarity index 84%
rename from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AggregateType.java
index 21ed3e2..830ab89 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/AggregateType.java
@@ -17,10 +17,9 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.group;
 
-public enum EnumFreeSlotPolicy {
-    SMALLEST_FIT,
-    LAST_FIT,
-    BIGGEST_FIT,
+public enum AggregateType {
+    PARTIAL,
+    FINAL
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/FrameToolsForGroupers.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/FrameToolsForGroupers.java
deleted file mode 100644
index 7416677..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/FrameToolsForGroupers.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * 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 org.apache.hyracks.dataflow.std.group;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hyracks.api.comm.FrameHelper;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-/**
- *
- */
-public class FrameToolsForGroupers {
-
-    public static void writeFields(byte[] buf, int offset, int length, ArrayTupleBuilder tupleBuilder)
-            throws HyracksDataException {
-        writeFields(buf, offset, length, tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
-                tupleBuilder.getSize());
-    }
-
-    public static void writeFields(byte[] buf, int offset, int length, int[] fieldsOffset, byte[] data, int dataOffset,
-            int dataLength) throws HyracksDataException {
-        if (dataLength + 4 * fieldsOffset.length > length) {
-            throw new HyracksDataException("Out of buffer bound: try to write too much data (" + dataLength
-                    + ") to the given bound (" + length + ").");
-        }
-
-        ByteBuffer buffer = ByteBuffer.wrap(buf, offset, length);
-        for (int i = 0; i < fieldsOffset.length; i++) {
-            buffer.putInt(fieldsOffset[i]);
-        }
-        buffer.put(data, dataOffset, dataLength);
-    }
-
-    public static void updateFrameMetaForNewTuple(ByteBuffer buffer, int addedTupleLength) throws HyracksDataException {
-        int currentTupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()));
-        int currentTupleEndOffset = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()) - 4
-                * currentTupleCount);
-        int newTupleEndOffset = currentTupleEndOffset + addedTupleLength;
-
-        // update tuple end offset
-        buffer.putInt(FrameHelper.getTupleCountOffset(buffer.capacity()) - 4 * (currentTupleCount + 1),
-                newTupleEndOffset);
-        // Update the tuple count
-        buffer.putInt(FrameHelper.getTupleCountOffset(buffer.capacity()), currentTupleCount + 1);
-    }
-
-    public static void updateFrameMetaForNewTuple(ByteBuffer buffer, int addedTupleLength, boolean isReset)
-            throws HyracksDataException {
-        int currentTupleCount;
-        int currentTupleEndOffset;
-        if (isReset) {
-            currentTupleCount = 0;
-            currentTupleEndOffset = 0;
-        } else {
-            currentTupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()));
-            currentTupleEndOffset = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()) - 4
-                    * currentTupleCount);
-        }
-        int newTupleEndOffset = currentTupleEndOffset + addedTupleLength;
-
-        // update tuple end offset
-        buffer.putInt(FrameHelper.getTupleCountOffset(buffer.capacity()) - 4 * (currentTupleCount + 1),
-                newTupleEndOffset);
-        // Update the tuple count
-        buffer.putInt(FrameHelper.getTupleCountOffset(buffer.capacity()), currentTupleCount + 1);
-    }
-
-    public static boolean isFrameOverflowing(ByteBuffer buffer, int length, boolean isReset)
-            throws HyracksDataException {
-
-        int currentTupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()));
-        if (currentTupleCount == 0 || isReset) {
-            return length + 4 + 4 > buffer.capacity();
-        }
-        int currentTupleEndOffset = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()) - 4
-                * currentTupleCount);
-        return currentTupleEndOffset + length + 4 + (currentTupleCount + 1) * 4 > buffer.capacity();
-    }
-}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index 7fbde54..44ddf44 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -16,94 +16,70 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package org.apache.hyracks.dataflow.std.group;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.BitSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
-import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFamily;
+import org.apache.hyracks.dataflow.std.buffermanager.IPartitionedTupleBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.ITuplePointerAccessor;
+import org.apache.hyracks.dataflow.std.buffermanager.PreferToSpillFullyOccupiedFramePolicy;
+import org.apache.hyracks.dataflow.std.buffermanager.VPartitionTupleBufferManager;
 import org.apache.hyracks.dataflow.std.structures.ISerializableTable;
 import org.apache.hyracks.dataflow.std.structures.SerializableHashTable;
 import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+import org.apache.hyracks.dataflow.std.util.FrameTuplePairComparator;
 
 public class HashSpillableTableFactory implements ISpillableTableFactory {
 
+    private static Logger LOGGER = Logger.getLogger(HashSpillableTableFactory.class.getName());
+    private static final double FUDGE_FACTOR = 1.1;
     private static final long serialVersionUID = 1L;
-    private final ITuplePartitionComputerFactory tpcf;
-    private final int tableSize;
+    private final IBinaryHashFunctionFamily[] hashFunctionFamilies;
 
-    public HashSpillableTableFactory(ITuplePartitionComputerFactory tpcf, int tableSize) {
-        this.tpcf = tpcf;
-        this.tableSize = tableSize;
+    public HashSpillableTableFactory(IBinaryHashFunctionFamily[] hashFunctionFamilies) {
+        this.hashFunctionFamilies = hashFunctionFamilies;
     }
 
-    /*
-     * (non-Javadoc)
-     *
-     * @see
-     * org.apache.hyracks.dataflow.std.aggregations.ISpillableTableFactory#
-     * buildSpillableTable(org.apache.hyracks.api.context.IHyracksTaskContext,
-     * int[], org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory[],
-     * org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory,
-     * edu.
-     * uci.ics.hyracks.dataflow.std.aggregations.IFieldAggregateDescriptorFactory
-     * [], org.apache.hyracks.api.dataflow.value.RecordDescriptor,
-     * org.apache.hyracks.api.dataflow.value.RecordDescriptor, int)
-     */
     @Override
-    public ISpillableTable buildSpillableTable(final IHyracksTaskContext ctx, final int[] keyFields,
-            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
-            IAggregatorDescriptorFactory aggregateFactory, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, final int framesLimit) throws HyracksDataException {
-        final int[] storedKeys = new int[keyFields.length];
-        @SuppressWarnings("rawtypes")
-        ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[keyFields.length];
+    public ISpillableTable buildSpillableTable(final IHyracksTaskContext ctx, int suggestTableSize, long dataBytesSize,
+            final int[] keyFields, final IBinaryComparator[] comparators,
+            final INormalizedKeyComputer firstKeyNormalizerFactory, IAggregatorDescriptorFactory aggregateFactory,
+            RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, final int framesLimit,
+            final int seed) throws HyracksDataException {
+        if (framesLimit < 2) {
+            throw new HyracksDataException("The frame limit is too small to partition the data");
+        }
+        final int tableSize = suggestTableSize;
+
+        final int[] intermediateResultKeys = new int[keyFields.length];
         for (int i = 0; i < keyFields.length; i++) {
-            storedKeys[i] = i;
-            storedKeySerDeser[i] = inRecordDescriptor.getFields()[keyFields[i]];
+            intermediateResultKeys[i] = i;
         }
 
-        RecordDescriptor internalRecordDescriptor = outRecordDescriptor;
-        final FrameTupleAccessor storedKeysAccessor1 = new FrameTupleAccessor(internalRecordDescriptor);
-        final FrameTupleAccessor storedKeysAccessor2 = new FrameTupleAccessor(internalRecordDescriptor);
+        final FrameTuplePairComparator ftpcInputCompareToAggregate = new FrameTuplePairComparator(keyFields,
+                intermediateResultKeys, comparators);
 
-        final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
-
-        final FrameTuplePairComparator ftpcPartial = new FrameTuplePairComparator(keyFields, storedKeys, comparators);
-
-        final FrameTuplePairComparator ftpcTuple = new FrameTuplePairComparator(storedKeys, storedKeys, comparators);
-
-        final ITuplePartitionComputer tpc = tpcf.createPartitioner();
-
-        final INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory
-                .createNormalizedKeyComputer();
-
-        int[] keyFieldsInPartialResults = new int[keyFields.length];
-        for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
-            keyFieldsInPartialResults[i] = i;
-        }
+        final ITuplePartitionComputer tpc = new FieldHashPartitionComputerFamily(keyFields, hashFunctionFamilies)
+                .createPartitioner(seed);
 
         final IAggregatorDescriptor aggregator = aggregateFactory.createAggregator(ctx, inRecordDescriptor,
-                outRecordDescriptor, keyFields, keyFieldsInPartialResults, null);
+                outRecordDescriptor, keyFields, intermediateResultKeys, null);
 
         final AggregateState aggregateState = aggregator.createAggregateStates();
 
@@ -114,365 +90,173 @@
             stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length + 1);
         }
 
+        //TODO(jf) research on the optimized partition size
+        final int numPartitions = getNumOfPartitions((int) (dataBytesSize / ctx.getInitialFrameSize()),
+                framesLimit - 1);
+        final int entriesPerPartition = (int) Math.ceil(1.0 * tableSize / numPartitions);
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("create hashtable, table size:" + tableSize + " file size:" + dataBytesSize + "  partitions:"
+                    + numPartitions);
+        }
+
         final ArrayTupleBuilder outputTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
 
+        final ISerializableTable hashTableForTuplePointer = new SerializableHashTable(tableSize, ctx);
+
         return new ISpillableTable() {
 
-            private int lastBufIndex;
+            private final TuplePointer pointer = new TuplePointer();
+            private final BitSet spilledSet = new BitSet(numPartitions);
+            final IPartitionedTupleBufferManager bufferManager = new VPartitionTupleBufferManager(ctx,
+                    PreferToSpillFullyOccupiedFramePolicy.createAtMostOneFrameForSpilledPartitionConstrain(spilledSet),
+                    numPartitions, framesLimit * ctx.getInitialFrameSize());
 
-            private IFrame outputFrame;
-            private FrameTupleAppender outputAppender;
+            final ITuplePointerAccessor bufferAccessor = bufferManager.getTupleAccessor(outRecordDescriptor);
 
-            private FrameTupleAppender stateAppender = new FrameTupleAppender();
+            private final PreferToSpillFullyOccupiedFramePolicy spillPolicy = new PreferToSpillFullyOccupiedFramePolicy(
+                    bufferManager, spilledSet, ctx.getInitialFrameSize());
 
-            private final ISerializableTable table = new SerializableHashTable(tableSize, ctx);
-            private final TuplePointer storedTuplePointer = new TuplePointer();
-            private final List<IFrame> frames = new ArrayList<>();
-
-            /**
-             * A tuple is "pointed" to by 3 entries in the tPointers array. [0]
-             * = Frame index in the "Frames" list, [1] = Tuple index in the
-             * frame, [2] = Poor man's normalized key for the tuple.
-             */
-            private int[] tPointers;
+            private final FrameTupleAppender outputAppender = new FrameTupleAppender(new VSizeFrame(ctx));
 
             @Override
-            public void sortFrames() throws HyracksDataException {
-                int sfIdx = storedKeys[0];
-                int totalTCount = table.getTupleCount();
-                tPointers = new int[totalTCount * 3];
-                int ptr = 0;
-
-                for (int i = 0; i < tableSize; i++) {
-                    int entry = i;
-                    int offset = 0;
-                    do {
-                        table.getTuplePointer(entry, offset, storedTuplePointer);
-                        if (storedTuplePointer.frameIndex < 0)
-                            break;
-                        tPointers[ptr * 3] = entry;
-                        tPointers[ptr * 3 + 1] = offset;
-                        table.getTuplePointer(entry, offset, storedTuplePointer);
-                        int fIndex = storedTuplePointer.frameIndex;
-                        int tIndex = storedTuplePointer.tupleIndex;
-                        storedKeysAccessor1.reset(frames.get(fIndex).getBuffer());
-                        int tStart = storedKeysAccessor1.getTupleStartOffset(tIndex);
-                        int f0StartRel = storedKeysAccessor1.getFieldStartOffset(tIndex, sfIdx);
-                        int f0EndRel = storedKeysAccessor1.getFieldEndOffset(tIndex, sfIdx);
-                        int f0Start = f0StartRel + tStart + storedKeysAccessor1.getFieldSlotsLength();
-                        tPointers[ptr * 3 + 2] = nkc == null ? 0 : nkc.normalize(storedKeysAccessor1.getBuffer()
-                                .array(), f0Start, f0EndRel - f0StartRel);
-                        ptr++;
-                        offset++;
-                    } while (true);
-                }
-                /**
-                 * Sort using quick sort
-                 */
-                if (tPointers.length > 0) {
-                    sort(tPointers, 0, totalTCount);
-                }
-            }
-
-            @Override
-            public void reset() {
-                lastBufIndex = -1;
-                tPointers = null;
-                table.reset();
-                aggregator.reset();
-            }
-
-            @Override
-            public boolean insert(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
-                if (lastBufIndex < 0)
-                    nextAvailableFrame();
-                int entry = tpc.partition(accessor, tIndex, tableSize);
-                boolean foundGroup = false;
-                int offset = 0;
-                do {
-                    table.getTuplePointer(entry, offset++, storedTuplePointer);
-                    if (storedTuplePointer.frameIndex < 0)
-                        break;
-                    storedKeysAccessor1.reset(frames.get(storedTuplePointer.frameIndex).getBuffer());
-                    int c = ftpcPartial.compare(accessor, tIndex, storedKeysAccessor1, storedTuplePointer.tupleIndex);
-                    if (c == 0) {
-                        foundGroup = true;
-                        break;
-                    }
-                } while (true);
-
-                if (!foundGroup) {
-
-                    stateTupleBuilder.reset();
-
-                    for (int k = 0; k < keyFields.length; k++) {
-                        stateTupleBuilder.addField(accessor, tIndex, keyFields[k]);
-                    }
-
-                    aggregator.init(stateTupleBuilder, accessor, tIndex, aggregateState);
-                    if (!stateAppender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
-                            stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
-                        if (!nextAvailableFrame()) {
-                            return false;
-                        }
-                        if (!stateAppender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
-                                stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
-                            throw new HyracksDataException("Cannot init external aggregate state in a frame.");
-                        }
-                    }
-
-                    storedTuplePointer.frameIndex = lastBufIndex;
-                    storedTuplePointer.tupleIndex = stateAppender.getTupleCount() - 1;
-                    table.insert(entry, storedTuplePointer);
-                } else {
-
-                    aggregator.aggregate(accessor, tIndex, storedKeysAccessor1, storedTuplePointer.tupleIndex,
-                            aggregateState);
-
-                }
-                return true;
-            }
-
-            @Override
-            public List<IFrame> getFrames() {
-                return frames;
-            }
-
-            @Override
-            public int getFrameCount() {
-                return lastBufIndex;
-            }
-
-            @Override
-            public void flushFrames(IFrameWriter writer, boolean isPartial) throws HyracksDataException {
-                if (outputFrame == null) {
-                    outputFrame = new VSizeFrame(ctx);
-                }
-
-                if (outputAppender == null) {
-                    outputAppender = new FrameTupleAppender();
-                }
-
-                outputAppender.reset(outputFrame, true);
-
-                if (tPointers == null) {
-                    // Not sorted
-                    for (int i = 0; i < tableSize; ++i) {
-                        int entry = i;
-                        int offset = 0;
-                        do {
-                            table.getTuplePointer(entry, offset++, storedTuplePointer);
-                            if (storedTuplePointer.frameIndex < 0)
-                                break;
-                            int bIndex = storedTuplePointer.frameIndex;
-                            int tIndex = storedTuplePointer.tupleIndex;
-
-                            storedKeysAccessor1.reset(frames.get(bIndex).getBuffer());
-
-                            outputTupleBuilder.reset();
-                            for (int k = 0; k < storedKeys.length; k++) {
-                                outputTupleBuilder.addField(storedKeysAccessor1, tIndex, storedKeys[k]);
-                            }
-
-                            if (isPartial) {
-
-                                aggregator.outputPartialResult(outputTupleBuilder, storedKeysAccessor1, tIndex,
-                                        aggregateState);
-
-                            } else {
-
-                                aggregator.outputFinalResult(outputTupleBuilder, storedKeysAccessor1, tIndex,
-                                        aggregateState);
-                            }
-
-                            if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
-                                    outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
-                                outputAppender.write(writer, true);
-                                if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
-                                        outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
-                                    throw new HyracksDataException(
-                                            "The output item is too large to be fit into a frame.");
-                                }
-                            }
-
-                        } while (true);
-                    }
-                    outputAppender.write(writer, true);
-                    aggregator.close();
-                    return;
-                }
-                int n = tPointers.length / 3;
-                for (int ptr = 0; ptr < n; ptr++) {
-                    int tableIndex = tPointers[ptr * 3];
-                    int rowIndex = tPointers[ptr * 3 + 1];
-                    table.getTuplePointer(tableIndex, rowIndex, storedTuplePointer);
-                    int frameIndex = storedTuplePointer.frameIndex;
-                    int tupleIndex = storedTuplePointer.tupleIndex;
-                    // Get the frame containing the value
-                    IFrame buffer = frames.get(frameIndex);
-                    storedKeysAccessor1.reset(buffer.getBuffer());
-
-                    outputTupleBuilder.reset();
-                    for (int k = 0; k < storedKeys.length; k++) {
-                        outputTupleBuilder.addField(storedKeysAccessor1, tupleIndex, storedKeys[k]);
-                    }
-
-                    if (isPartial) {
-
-                        aggregator.outputPartialResult(outputTupleBuilder, storedKeysAccessor1, tupleIndex,
-                                aggregateState);
-
-                    } else {
-
-                        aggregator.outputFinalResult(outputTupleBuilder, storedKeysAccessor1, tupleIndex,
-                                aggregateState);
-                    }
-
-                    if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
-                            outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
-                        outputAppender.write(writer, true);
-                        if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
-                                outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
-                            throw new HyracksDataException("The output item is too large to be fit into a frame.");
-                        }
-                    }
-                }
-                outputAppender.write(writer, true);
+            public void close() throws HyracksDataException {
+                hashTableForTuplePointer.close();
                 aggregator.close();
             }
 
             @Override
-            public void close() {
-                lastBufIndex = -1;
-                tPointers = null;
-                table.close();
-                frames.clear();
-                aggregateState.close();
+            public void clear(int partition) throws HyracksDataException {
+                for (int p = getFirstEntryInHashTable(partition); p < getLastEntryInHashTable(partition); p++) {
+                    hashTableForTuplePointer.delete(p);
+                }
+                bufferManager.clearPartition(partition);
             }
 
-            /**
-             * Set the working frame to the next available frame in the frame
-             * list. There are two cases:<br>
-             * 1) If the next frame is not initialized, allocate a new frame. 2)
-             * When frames are already created, they are recycled.
-             *
-             * @return Whether a new frame is added successfully.
-             * @throws HyracksDataException
-             */
-            private boolean nextAvailableFrame() throws HyracksDataException {
-                // Return false if the number of frames is equal to the limit.
-                if (lastBufIndex + 1 >= framesLimit)
-                    return false;
+            private int getPartition(int entryInHashTable) {
+                return entryInHashTable / entriesPerPartition;
+            }
 
-                if (frames.size() < framesLimit) {
-                    // Insert a new frame
-                    IFrame frame = new VSizeFrame(ctx);
-                    frames.add(frame);
-                    stateAppender.reset(frame, true);
-                    lastBufIndex = frames.size() - 1;
-                } else {
-                    // Reuse an old frame
-                    lastBufIndex++;
-                    stateAppender.reset(frames.get(lastBufIndex), true);
+            private int getFirstEntryInHashTable(int partition) {
+                return partition * entriesPerPartition;
+            }
+
+            private int getLastEntryInHashTable(int partition) {
+                return Math.min(tableSize, (partition + 1) * entriesPerPartition);
+            }
+
+            @Override
+            public boolean insert(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+                int entryInHashTable = tpc.partition(accessor, tIndex, tableSize);
+                for (int i = 0; i < hashTableForTuplePointer.getTupleCount(entryInHashTable); i++) {
+                    hashTableForTuplePointer.getTuplePointer(entryInHashTable, i, pointer);
+                    bufferAccessor.reset(pointer);
+                    int c = ftpcInputCompareToAggregate.compare(accessor, tIndex, bufferAccessor);
+                    if (c == 0) {
+                        aggregateExistingTuple(accessor, tIndex, bufferAccessor, pointer.tupleIndex);
+                        return true;
+                    }
                 }
+
+                return insertNewAggregateEntry(entryInHashTable, accessor, tIndex);
+            }
+
+            private boolean insertNewAggregateEntry(int entryInHashTable, IFrameTupleAccessor accessor, int tIndex)
+                    throws HyracksDataException {
+                initStateTupleBuilder(accessor, tIndex);
+                int pid = getPartition(entryInHashTable);
+
+                if (!bufferManager.insertTuple(pid, stateTupleBuilder.getByteArray(),
+                        stateTupleBuilder.getFieldEndOffsets(), 0, stateTupleBuilder.getSize(), pointer)) {
+                    return false;
+                }
+                hashTableForTuplePointer.insert(entryInHashTable, pointer);
                 return true;
             }
 
-            private void sort(int[] tPointers, int offset, int length) throws HyracksDataException {
-                int m = offset + (length >> 1);
-                int mTable = tPointers[m * 3];
-                int mRow = tPointers[m * 3 + 1];
-                int mNormKey = tPointers[m * 3 + 2];
+            private void initStateTupleBuilder(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+                stateTupleBuilder.reset();
+                for (int k = 0; k < keyFields.length; k++) {
+                    stateTupleBuilder.addField(accessor, tIndex, keyFields[k]);
+                }
+                aggregator.init(stateTupleBuilder, accessor, tIndex, aggregateState);
+            }
 
-                table.getTuplePointer(mTable, mRow, storedTuplePointer);
-                int mFrame = storedTuplePointer.frameIndex;
-                int mTuple = storedTuplePointer.tupleIndex;
-                storedKeysAccessor1.reset(frames.get(mFrame).getBuffer());
+            private void aggregateExistingTuple(IFrameTupleAccessor accessor, int tIndex,
+                    ITuplePointerAccessor bufferAccessor, int tupleIndex) throws HyracksDataException {
+                aggregator.aggregate(accessor, tIndex, bufferAccessor, tupleIndex, aggregateState);
+            }
 
-                int a = offset;
-                int b = a;
-                int c = offset + length - 1;
-                int d = c;
-                while (true) {
-                    while (b <= c) {
-                        int bTable = tPointers[b * 3];
-                        int bRow = tPointers[b * 3 + 1];
-                        int bNormKey = tPointers[b * 3 + 2];
-                        int cmp = 0;
-                        if (bNormKey != mNormKey) {
-                            cmp = ((((long) bNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1 : 1;
-                        } else {
-                            table.getTuplePointer(bTable, bRow, storedTuplePointer);
-                            int bFrame = storedTuplePointer.frameIndex;
-                            int bTuple = storedTuplePointer.tupleIndex;
-                            storedKeysAccessor2.reset(frames.get(bFrame).getBuffer());
-                            cmp = ftpcTuple.compare(storedKeysAccessor2, bTuple, storedKeysAccessor1, mTuple);
+            @Override
+            public int flushFrames(int partition, IFrameWriter writer, AggregateType type) throws HyracksDataException {
+                int count = 0;
+                for (int hashEntryPid = getFirstEntryInHashTable(partition); hashEntryPid < getLastEntryInHashTable(
+                        partition); hashEntryPid++) {
+                    count += hashTableForTuplePointer.getTupleCount(hashEntryPid);
+                    for (int tid = 0; tid < hashTableForTuplePointer.getTupleCount(hashEntryPid); tid++) {
+                        hashTableForTuplePointer.getTuplePointer(hashEntryPid, tid, pointer);
+                        bufferAccessor.reset(pointer);
+                        outputTupleBuilder.reset();
+                        for (int k = 0; k < intermediateResultKeys.length; k++) {
+                            outputTupleBuilder.addField(bufferAccessor.getBuffer().array(),
+                                    bufferAccessor.getAbsFieldStartOffset(intermediateResultKeys[k]),
+                                    bufferAccessor.getFieldLength(intermediateResultKeys[k]));
                         }
-                        if (cmp > 0) {
-                            break;
+
+                        boolean hasOutput = false;
+                        switch (type) {
+                            case PARTIAL:
+                                hasOutput = aggregator.outputPartialResult(outputTupleBuilder, bufferAccessor,
+                                        pointer.tupleIndex, aggregateState);
+                                break;
+                            case FINAL:
+                                hasOutput = aggregator.outputFinalResult(outputTupleBuilder, bufferAccessor,
+                                        pointer.tupleIndex, aggregateState);
+                                break;
                         }
-                        if (cmp == 0) {
-                            swap(tPointers, a++, b);
+
+                        if (hasOutput && !outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
+                                outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
+                            outputAppender.write(writer, true);
+                            if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
+                                    outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
+                                throw new HyracksDataException("The output item is too large to be fit into a frame.");
+                            }
                         }
-                        ++b;
                     }
-                    while (c >= b) {
-                        int cTable = tPointers[c * 3];
-                        int cRow = tPointers[c * 3 + 1];
-                        int cNormKey = tPointers[c * 3 + 2];
-                        int cmp = 0;
-                        if (cNormKey != mNormKey) {
-                            cmp = ((((long) cNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1 : 1;
-                        } else {
-                            table.getTuplePointer(cTable, cRow, storedTuplePointer);
-                            int cFrame = storedTuplePointer.frameIndex;
-                            int cTuple = storedTuplePointer.tupleIndex;
-                            storedKeysAccessor2.reset(frames.get(cFrame).getBuffer());
-                            cmp = ftpcTuple.compare(storedKeysAccessor2, cTuple, storedKeysAccessor1, mTuple);
-                        }
-                        if (cmp < 0) {
-                            break;
-                        }
-                        if (cmp == 0) {
-                            swap(tPointers, c, d--);
-                        }
-                        --c;
-                    }
-                    if (b > c)
-                        break;
-                    swap(tPointers, b++, c--);
                 }
-
-                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 ((s = d - c) > 1) {
-                    sort(tPointers, n - s, s);
-                }
+                outputAppender.write(writer, true);
+                spilledSet.set(partition);
+                return count;
             }
 
-            private void swap(int x[], int a, int b) {
-                for (int i = 0; i < 3; ++i) {
-                    int t = x[a * 3 + i];
-                    x[a * 3 + i] = x[b * 3 + i];
-                    x[b * 3 + i] = t;
-                }
+            @Override
+            public int getNumPartitions() {
+                return bufferManager.getNumPartitions();
             }
 
-            private void vecswap(int x[], int a, int b, int n) {
-                for (int i = 0; i < n; i++, a++, b++) {
-                    swap(x, a, b);
-                }
+            @Override
+            public int findVictimPartition(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+                int entryInHashTable = tpc.partition(accessor, tIndex, tableSize);
+                int partition = getPartition(entryInHashTable);
+                return spillPolicy.selectVictimPartition(partition);
             }
-
         };
     }
 
+    private int getNumOfPartitions(int nubmerOfFramesForData, int frameLimit) {
+        if (frameLimit > nubmerOfFramesForData) {
+            return 1; // all in memory, we will create a big partition
+        }
+        int numberOfPartitions = (int) (Math
+                .ceil((nubmerOfFramesForData * FUDGE_FACTOR - frameLimit) / (frameLimit - 1)));
+        if (numberOfPartitions <= 0) {
+            numberOfPartitions = 1; //becomes in-memory hash
+        }
+        if (numberOfPartitions > frameLimit) {
+            numberOfPartitions = (int) Math.ceil(Math.sqrt(nubmerOfFramesForData * FUDGE_FACTOR));
+            return Math.max(2, Math.min(numberOfPartitions, frameLimit));
+        }
+        return numberOfPartitions;
+    }
+
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IAggregatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IAggregatorDescriptor.java
index 4607032..9552294 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IAggregatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IAggregatorDescriptor.java
@@ -29,21 +29,18 @@
      *
      * @return
      */
-    public AggregateState createAggregateStates();
+    AggregateState createAggregateStates();
 
     /**
      * Initialize the state based on the input tuple.
-     *
+     * 
+     * @param tupleBuilder
      * @param accessor
      * @param tIndex
-     * @param fieldOutput
-     *            The data output for the frame containing the state. This may
-     *            be null, if the state is maintained as a java object
      * @param state
-     *            The state to be initialized.
      * @throws HyracksDataException
      */
-    public void init(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex, AggregateState state)
+    void init(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex, AggregateState state)
             throws HyracksDataException;
 
     /**
@@ -51,60 +48,58 @@
      * too. Note that here the frame is not an input argument, since it can be
      * reset outside of the aggregator (simply reset the starting index of the
      * buffer).
-     *
-     * @param state
      */
-    public void reset();
+    void reset();
 
     /**
      * Aggregate the value. Aggregate state should be updated correspondingly.
      *
      * @param accessor
      * @param tIndex
-     * @param data
+     * @param stateAccessor
      *            The buffer containing the state, if frame-based-state is used.
      *            This means that it can be null if java-object-based-state is
      *            used.
-     * @param offset
+     * @param stateTupleIndex
      * @param state
      *            The aggregate state.
      * @throws HyracksDataException
      */
-    public void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor,
-            int stateTupleIndex, AggregateState state) throws HyracksDataException;
+    void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor, int stateTupleIndex,
+            AggregateState state) throws HyracksDataException;
 
     /**
      * Output the partial aggregation result.
      *
-     * @param fieldOutput
-     *            The data output for the output frame
-     * @param data
-     *            The buffer containing the aggregation state
-     * @param offset
+     * @param tupleBuilder
+     *            The data output for the output aggregation result
+     * @param stateAccessor
+     *            The stateAccessor buffer containing the aggregation state
+     * @param tIndex
      * @param state
      *            The aggregation state.
-     * @return TODO
+     * @return true if it has any output writed to {@code tupleBuilder}
      * @throws HyracksDataException
      */
-    public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+    boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor stateAccessor, int tIndex,
             AggregateState state) throws HyracksDataException;
 
     /**
      * Output the final aggregation result.
      *
-     * @param fieldOutput
+     * @param tupleBuilder
      *            The data output for the output frame
-     * @param data
+     * @param stateAccessor
      *            The buffer containing the aggregation state
-     * @param offset
+     * @param tIndex
      * @param state
      *            The aggregation state.
-     * @return TODO
+     * @return true if it has any output writed to {@code tupleBuilder}
      * @throws HyracksDataException
      */
-    public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+    boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor stateAccessor, int tIndex,
             AggregateState state) throws HyracksDataException;
 
-    public void close();
+    void close();
 
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTable.java
index 1f99183..2b9ad54 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTable.java
@@ -18,26 +18,56 @@
  */
 package org.apache.hyracks.dataflow.std.group;
 
-import java.util.List;
-
-import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public interface ISpillableTable {
 
-    public void close();
+    /**
+     * Release all the storage resources.
+     * @throws HyracksDataException
+     */
+    void close() throws HyracksDataException;
 
-    public void reset();
+    /**
+     * Reset the specific partition to the initial state. The occupied resources will be released.
+     * @param partition
+     * @throws HyracksDataException
+     */
+    void clear(int partition) throws HyracksDataException;
 
-    public int getFrameCount();
+    /**
+     * Insert the specific tuple into the table.
+     * @param accessor
+     * @param tIndex
+     * @return
+     * @throws HyracksDataException
+     */
+    boolean insert(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
 
-    public List<IFrame> getFrames();
+    /**
+     * Flush the certain partition to writer, and return the numOfTuples that have been flushed
+     * @param partition
+     * @param writer
+     * @param type
+     * @return
+     * @throws HyracksDataException
+     */
+    int flushFrames(int partition, IFrameWriter writer, AggregateType type) throws HyracksDataException;
 
-    public void sortFrames() throws HyracksDataException;
+    /**
+     * Get number of partitions
+     */
+    int getNumPartitions();
 
-    public boolean insert(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
-
-    public void flushFrames(IFrameWriter writer, boolean isPartial) throws HyracksDataException;
+    /**
+     * When the table is full, it will return a proper partition which will be the flush() candidate.
+     * The {@code accessor} and {@code tIndex} given the reference to the tuple to be inserted.
+     * @return the partition id of the victim, -1 if it failed to find a partition
+     * @param accessor
+     * @param tIndex
+     */
+    int findVictimPartition(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTableFactory.java
index 6b90b37..dbe6858 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTableFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/ISpillableTableFactory.java
@@ -21,15 +21,15 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface ISpillableTableFactory extends Serializable {
-    ISpillableTable buildSpillableTable(IHyracksTaskContext ctx, int[] keyFields,
-            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory normalizedKeyComputerFactory,
+    ISpillableTable buildSpillableTable(IHyracksTaskContext ctx, int inputSizeInTuple, long dataBytesSize, int[] keyFields,
+            IBinaryComparator[] comparatorFactories, INormalizedKeyComputer firstKeyNormalizerFactory,
             IAggregatorDescriptorFactory aggregateFactory, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, int framesLimit) throws HyracksDataException;
+            RecordDescriptor outRecordDescriptor, int framesLimit, int seed) throws HyracksDataException;
 
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java
index b25d16c..e326f39 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java
@@ -80,15 +80,15 @@
             }
 
             @Override
-            public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor,
+            public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor stateAccessor,
                     int tIndex, AggregateState state) throws HyracksDataException {
                 DataOutput dos = tupleBuilder.getDataOutput();
 
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int tupleOffset = stateAccessor.getTupleStartOffset(tIndex);
                 for (int i = 0; i < aggregators.length; i++) {
-                    int fieldOffset = accessor.getFieldStartOffset(tIndex, keys.length + i);
-                    aggregators[i].outputPartialResult(dos, accessor.getBuffer().array(),
-                            fieldOffset + accessor.getFieldSlotsLength() + tupleOffset,
+                    int fieldOffset = stateAccessor.getFieldStartOffset(tIndex, keys.length + i);
+                    aggregators[i].outputPartialResult(dos, stateAccessor.getBuffer().array(),
+                            fieldOffset + stateAccessor.getFieldSlotsLength() + tupleOffset,
                             ((AggregateState[]) state.state)[i]);
                     tupleBuilder.addFieldEndOffset();
                 }
@@ -96,16 +96,16 @@
             }
 
             @Override
-            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            public boolean outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor stateAccessor, int tIndex,
                     AggregateState state) throws HyracksDataException {
                 DataOutput dos = tupleBuilder.getDataOutput();
 
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int tupleOffset = stateAccessor.getTupleStartOffset(tIndex);
                 for (int i = 0; i < aggregators.length; i++) {
                     if (aggregators[i].needsBinaryState()) {
-                        int fieldOffset = accessor.getFieldStartOffset(tIndex, keys.length + i);
-                        aggregators[i].outputFinalResult(dos, accessor.getBuffer().array(),
-                                tupleOffset + accessor.getFieldSlotsLength() + fieldOffset,
+                        int fieldOffset = stateAccessor.getFieldStartOffset(tIndex, keys.length + i);
+                        aggregators[i].outputFinalResult(dos, stateAccessor.getBuffer().array(),
+                                tupleOffset + stateAccessor.getFieldSlotsLength() + fieldOffset,
                                 ((AggregateState[]) state.state)[i]);
                     } else {
                         aggregators[i].outputFinalResult(dos, null, 0, ((AggregateState[]) state.state)[i]);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
index c499749..ba3853c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
@@ -18,125 +18,123 @@
  */
 package org.apache.hyracks.dataflow.std.group.external;
 
-import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.LinkedList;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.io.RunFileReader;
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
 import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.ISpillableTable;
 import org.apache.hyracks.dataflow.std.group.ISpillableTableFactory;
 
-class ExternalGroupBuildOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
+public class ExternalGroupBuildOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable
+        implements IRunFileWriterGenerator {
+
+    private static Logger LOGGER = Logger.getLogger("ExternalGroupBuildPhase");
     private final IHyracksTaskContext ctx;
     private final Object stateId;
     private final int[] keyFields;
-    private final IBinaryComparatorFactory[] comparatorFactories;
-    private final INormalizedKeyComputerFactory firstNormalizerFactory;
+    private final IBinaryComparator[] comparators;
+    private final INormalizedKeyComputer firstNormalizerComputer;
     private final IAggregatorDescriptorFactory aggregatorFactory;
     private final int framesLimit;
     private final ISpillableTableFactory spillableTableFactory;
     private final RecordDescriptor inRecordDescriptor;
     private final RecordDescriptor outRecordDescriptor;
-    private final FrameTupleAccessor accessor;
+    private final int tableSize;
+    private final long fileSize;
 
+    private ExternalHashGroupBy externalGroupBy;
     private ExternalGroupState state;
+    private boolean isFailed = false;
 
-    ExternalGroupBuildOperatorNodePushable(IHyracksTaskContext ctx, Object stateId, int[] keyFields, int framesLimit,
-            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory firstNormalizerFactory,
-            IAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, ISpillableTableFactory spillableTableFactory) {
+    public ExternalGroupBuildOperatorNodePushable(IHyracksTaskContext ctx, Object stateId, int tableSize, long fileSize,
+            int[] keyFields, int framesLimit, IBinaryComparatorFactory[] comparatorFactories,
+            INormalizedKeyComputerFactory firstNormalizerFactory, IAggregatorDescriptorFactory aggregatorFactory,
+            RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor,
+            ISpillableTableFactory spillableTableFactory) {
         this.ctx = ctx;
         this.stateId = stateId;
         this.framesLimit = framesLimit;
         this.aggregatorFactory = aggregatorFactory;
         this.keyFields = keyFields;
-        this.comparatorFactories = comparatorFactories;
-        this.firstNormalizerFactory = firstNormalizerFactory;
+        this.comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        this.firstNormalizerComputer = firstNormalizerFactory.createNormalizedKeyComputer();
         this.spillableTableFactory = spillableTableFactory;
         this.inRecordDescriptor = inRecordDescriptor;
         this.outRecordDescriptor = outRecordDescriptor;
-        this.accessor = new FrameTupleAccessor(inRecordDescriptor);
+        this.tableSize = tableSize;
+        this.fileSize = fileSize;
     }
 
     @Override
     public void open() throws HyracksDataException {
         state = new ExternalGroupState(ctx.getJobletContext().getJobId(), stateId);
-        state.setRuns(new LinkedList<RunFileReader>());
-        ISpillableTable table = spillableTableFactory.buildSpillableTable(ctx, keyFields, comparatorFactories,
-                firstNormalizerFactory, aggregatorFactory, inRecordDescriptor, outRecordDescriptor, framesLimit);
-        table.reset();
+        ISpillableTable table = spillableTableFactory.buildSpillableTable(ctx, tableSize, fileSize, keyFields,
+                comparators, firstNormalizerComputer, aggregatorFactory, inRecordDescriptor, outRecordDescriptor,
+                framesLimit, 0);
+        RunFileWriter[] runFileWriters = new RunFileWriter[table.getNumPartitions()];
+        this.externalGroupBy = new ExternalHashGroupBy(this, table, runFileWriters, inRecordDescriptor);
+
         state.setSpillableTable(table);
+        state.setRuns(runFileWriters);
+        state.setSpilledNumTuples(externalGroupBy.getSpilledNumTuples());
     }
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        accessor.reset(buffer);
-        int tupleCount = accessor.getTupleCount();
-        ISpillableTable gTable = state.getSpillableTable();
-        for (int i = 0; i < tupleCount; i++) {
-            /**
-             * If the group table is too large, flush the table into
-             * a run file.
-             */
-            if (!gTable.insert(accessor, i)) {
-                flushFramesToRun();
-                if (!gTable.insert(accessor, i))
-                    throw new HyracksDataException("Failed to insert a new buffer into the aggregate operator!");
-            }
-        }
+        externalGroupBy.insert(buffer);
     }
 
     @Override
     public void fail() throws HyracksDataException {
-        //do nothing for failures
+        isFailed = true;
     }
 
     @Override
     public void close() throws HyracksDataException {
-        ISpillableTable gTable = state.getSpillableTable();
-        if (gTable.getFrameCount() >= 0) {
-            if (state.getRuns().size() > 0) {
-                /**
-                 * flush the memory into the run file.
-                 */
-                flushFramesToRun();
-                gTable.close();
-                gTable = null;
+        if (isFailed) {
+            for (int i = 0; i < state.getRuns().length; i++) {
+                RunFileWriter run = state.getRuns()[i];
+                if (run != null) {
+                    run.getFileReference().delete();
+                }
+            }
+        } else {
+            externalGroupBy.flushSpilledPartitions();
+            ctx.setStateObject(state);
+            if (LOGGER.isLoggable(Level.FINE)) {
+                int numOfPartition = state.getSpillableTable().getNumPartitions();
+                int numOfSpilledPart = 0;
+                for (int i = 0; i < numOfPartition; i++) {
+                    if (state.getSpilledNumTuples()[i] > 0) {
+                        numOfSpilledPart++;
+                    }
+                }
+                LOGGER.fine("level 0:" + "build with " + numOfPartition + " partitions" + ", spilled "
+                        + numOfSpilledPart + " partitions");
             }
         }
-        ctx.setStateObject(state);
+        state = null;
+        externalGroupBy = null;
     }
 
-    private void flushFramesToRun() throws HyracksDataException {
-        FileReference runFile;
-        try {
-            runFile = ctx.getJobletContext().createManagedWorkspaceFile(
-                    ExternalGroupOperatorDescriptor.class.getSimpleName());
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
-        RunFileWriter writer = new RunFileWriter(runFile, ctx.getIOManager());
-        writer.open();
-        ISpillableTable gTable = state.getSpillableTable();
-        try {
-            gTable.sortFrames();
-            gTable.flushFrames(writer, true);
-        } catch (Exception ex) {
-            throw new HyracksDataException(ex);
-        } finally {
-            writer.close();
-        }
-        gTable.reset();
-        state.getRuns().add(writer.createDeleteOnCloseReader());
+    @Override
+    public RunFileWriter getRunFileWriter() throws HyracksDataException {
+        FileReference file = ctx.getJobletContext()
+                .createManagedWorkspaceFile(ExternalGroupOperatorDescriptor.class.getSimpleName());
+        return new RunFileWriter(file, ctx.getIOManager());
     }
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
deleted file mode 100644
index 5c79014..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
+++ /dev/null
@@ -1,470 +0,0 @@
-/*
- * 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 org.apache.hyracks.dataflow.std.group.external;
-
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.IFrameTupleAccessor;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppenderAccessor;
-import org.apache.hyracks.dataflow.common.io.RunFileReader;
-import org.apache.hyracks.dataflow.common.io.RunFileWriter;
-import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-import org.apache.hyracks.dataflow.std.group.AggregateState;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptor;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
-import org.apache.hyracks.dataflow.std.group.ISpillableTable;
-import org.apache.hyracks.dataflow.std.util.ReferenceEntry;
-import org.apache.hyracks.dataflow.std.util.ReferencedPriorityQueue;
-
-class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
-    private final IHyracksTaskContext ctx;
-    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;
-    private final IAggregatorDescriptor aggregator;
-    private final boolean isOutputSorted;
-    private final int framesLimit;
-    private final RecordDescriptor outRecordDescriptor;
-    /**
-     * Input frames, one for each run file.
-     */
-    private List<IFrame> inFrames;
-    /**
-     * Output frame.
-     */
-    private IFrame outFrame, writerFrame;
-    private final FrameTupleAppenderAccessor outAppender;
-    private FrameTupleAppender writerAppender;
-    private LinkedList<RunFileReader> runs;
-    private ExternalGroupState aggState;
-    private ArrayTupleBuilder finalTupleBuilder;
-    /**
-     * how many frames to be read ahead once
-     */
-    private int runFrameLimit = 1;
-    private int[] currentFrameIndexInRun;
-    private int[] currentRunFrames;
-
-    ExternalGroupMergeOperatorNodePushable(IHyracksTaskContext ctx, Object stateId,
-            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory nmkFactory, int[] keyFields,
-            IAggregatorDescriptorFactory mergerFactory, boolean isOutputSorted, int framesLimit,
-            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
-        this.stateId = stateId;
-        this.keyFields = keyFields;
-        comparators = new IBinaryComparator[comparatorFactories.length];
-        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;
-        }
-
-        aggregator = mergerFactory.createAggregator(ctx, outRecordDescriptor, outRecordDescriptor, keyFields,
-                keyFieldsInPartialResults, writer);
-        aggregateState = aggregator.createAggregateStates();
-
-        storedKeys = new int[keyFields.length];
-        /**
-         * Get the list of the fields in the stored records.
-         */
-        for (int i = 0; i < keyFields.length; ++i) {
-            storedKeys[i] = i;
-        }
-
-        tupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
-        this.ctx = ctx;
-        outAppender = new FrameTupleAppenderAccessor(outRecordDescriptor);
-        this.isOutputSorted = isOutputSorted;
-        this.framesLimit = framesLimit;
-        this.outRecordDescriptor = outRecordDescriptor;
-    }
-
-    @Override
-    public void initialize() throws HyracksDataException {
-        aggState = (ExternalGroupState) ctx.getStateObject(stateId);
-        runs = aggState.getRuns();
-        try {
-            writer.open();
-            if (runs.size() <= 0) {
-                ISpillableTable gTable = aggState.getSpillableTable();
-                if (gTable != null) {
-                    if (isOutputSorted)
-                        gTable.sortFrames();
-                    gTable.flushFrames(writer, false);
-                }
-                gTable = null;
-                aggState = null;
-            } else {
-                aggState = null;
-                runs = new LinkedList<RunFileReader>(runs);
-                inFrames = new ArrayList<>();
-                outFrame = new VSizeFrame(ctx);
-                outAppender.reset(outFrame, true);
-                while (runs.size() > 0) {
-                    try {
-                        doPass(runs);
-                    } catch (Exception e) {
-                        throw new HyracksDataException(e);
-                    }
-                }
-                inFrames.clear();
-            }
-        } catch (Exception e) {
-            writer.fail();
-            throw new HyracksDataException(e);
-        } finally {
-            aggregateState.close();
-            writer.close();
-        }
-    }
-
-    private void doPass(LinkedList<RunFileReader> runs) throws HyracksDataException {
-        FileReference newRun = null;
-        IFrameWriter writer = this.writer;
-        boolean finalPass = false;
-
-        while (inFrames.size() + 2 < framesLimit) {
-            inFrames.add(new VSizeFrame(ctx));
-        }
-        int runNumber;
-        if (runs.size() + 2 <= framesLimit) {
-            finalPass = true;
-            runFrameLimit = (framesLimit - 2) / runs.size();
-            runNumber = runs.size();
-        } else {
-            runNumber = framesLimit - 2;
-            newRun = ctx.getJobletContext().createManagedWorkspaceFile(
-                    ExternalGroupOperatorDescriptor.class.getSimpleName());
-            writer = new RunFileWriter(newRun, ctx.getIOManager());
-            writer.open();
-        }
-        try {
-            currentFrameIndexInRun = new int[runNumber];
-            currentRunFrames = new int[runNumber];
-            /**
-             * Create file readers for each input run file, only for
-             * the ones fit into the inFrames
-             */
-            RunFileReader[] runFileReaders = new RunFileReader[runNumber];
-            FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
-            Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
-            ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(runNumber, comparator, keyFields,
-                    nmkComputer);
-            /**
-             * current tuple index in each run
-             */
-            int[] tupleIndices = new int[runNumber];
-
-            for (int i = 0; i < runNumber; i++) {
-                int runIndex = topTuples.peek().getRunid();
-                tupleIndices[runIndex] = 0;
-                // Load the run file
-                runFileReaders[runIndex] = runs.get(runIndex);
-                runFileReaders[runIndex].open();
-
-                currentRunFrames[runIndex] = 0;
-                currentFrameIndexInRun[runIndex] = runIndex * runFrameLimit;
-                for (int j = 0; j < runFrameLimit; j++) {
-                    int frameIndex = currentFrameIndexInRun[runIndex] + j;
-                    if (runFileReaders[runIndex].nextFrame(inFrames.get(frameIndex))) {
-                        tupleAccessors[frameIndex] = new FrameTupleAccessor(outRecordDescriptor);
-                        tupleAccessors[frameIndex].reset(inFrames.get(frameIndex).getBuffer());
-                        currentRunFrames[runIndex]++;
-                        if (j == 0)
-                            setNextTopTuple(runIndex, tupleIndices, runFileReaders, tupleAccessors, topTuples);
-                    } else {
-                        break;
-                    }
-                }
-            }
-
-            /**
-             * Start merging
-             */
-            while (!topTuples.areRunsExhausted()) {
-                /**
-                 * Get the top record
-                 */
-                ReferenceEntry top = topTuples.peek();
-                int tupleIndex = top.getTupleIndex();
-                int runIndex = topTuples.peek().getRunid();
-                IFrameTupleAccessor fta = top.getAccessor();
-
-                int currentTupleInOutFrame = outAppender.getTupleCount() - 1;
-                if (currentTupleInOutFrame < 0
-                        || compareFrameTuples(fta, tupleIndex, outAppender, currentTupleInOutFrame) != 0) {
-                    /**
-                     * Initialize the first output record Reset the
-                     * tuple builder
-                     */
-
-                    tupleBuilder.reset();
-
-                    for (int k = 0; k < storedKeys.length; k++) {
-                        tupleBuilder.addField(fta, tupleIndex, storedKeys[k]);
-                    }
-
-                    aggregator.init(tupleBuilder, fta, tupleIndex, aggregateState);
-
-                    if (!outAppender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(),
-                            tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
-                        flushOutFrame(writer, finalPass);
-                        if (!outAppender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(),
-                                tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
-                            throw new HyracksDataException(
-                                    "The partial result is too large to be initialized in a frame.");
-                        }
-                    }
-
-                } else {
-                    /**
-                     * if new tuple is in the same group of the
-                     * current aggregator do merge and output to the
-                     * outFrame
-                     */
-
-                    aggregator.aggregate(fta, tupleIndex, outAppender, currentTupleInOutFrame, aggregateState);
-
-                }
-                tupleIndices[runIndex]++;
-                setNextTopTuple(runIndex, tupleIndices, runFileReaders, tupleAccessors, topTuples);
-            }
-
-            if (outAppender.getTupleCount() > 0) {
-                flushOutFrame(writer, finalPass);
-                outAppender.reset(outFrame, true);
-            }
-
-            aggregator.close();
-
-            runs.subList(0, runNumber).clear();
-            /**
-             * insert the new run file into the beginning of the run
-             * file list
-             */
-            if (!finalPass) {
-                runs.add(0, ((RunFileWriter) writer).createDeleteOnCloseReader());
-            }
-        } finally {
-            if (!finalPass) {
-                writer.close();
-            }
-        }
-    }
-
-    private void flushOutFrame(IFrameWriter writer, boolean isFinal) throws HyracksDataException {
-
-        if (finalTupleBuilder == null) {
-            finalTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
-        }
-
-        if (writerFrame == null) {
-            writerFrame = new VSizeFrame(ctx);
-        }
-
-        if (writerAppender == null) {
-            writerAppender = new FrameTupleAppender();
-            writerAppender.reset(writerFrame, true);
-        }
-
-        for (int i = 0; i < outAppender.getTupleCount(); i++) {
-
-            finalTupleBuilder.reset();
-
-            for (int k = 0; k < storedKeys.length; k++) {
-                finalTupleBuilder.addField(outAppender, i, storedKeys[k]);
-            }
-
-            if (isFinal) {
-
-                aggregator.outputFinalResult(finalTupleBuilder, outAppender, i, aggregateState);
-
-            } else {
-
-                aggregator.outputPartialResult(finalTupleBuilder, outAppender, i, aggregateState);
-            }
-
-            if (!writerAppender.appendSkipEmptyField(finalTupleBuilder.getFieldEndOffsets(),
-                    finalTupleBuilder.getByteArray(), 0, finalTupleBuilder.getSize())) {
-                writerAppender.write(writer, true);
-                if (!writerAppender.appendSkipEmptyField(finalTupleBuilder.getFieldEndOffsets(),
-                        finalTupleBuilder.getByteArray(), 0, finalTupleBuilder.getSize())) {
-                    throw new HyracksDataException("Aggregation output is too large to be fit into a frame.");
-                }
-            }
-        }
-        writerAppender.write(writer, true);
-
-    }
-
-    private void setNextTopTuple(int runIndex, int[] tupleIndices, RunFileReader[] runCursors,
-            FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws HyracksDataException {
-        int runStart = runIndex * runFrameLimit;
-        boolean existNext = false;
-        if (tupleAccessors[currentFrameIndexInRun[runIndex]] == null || runCursors[runIndex] == null) {
-            /**
-             * run already closed
-             */
-            existNext = false;
-        } else if (currentFrameIndexInRun[runIndex] - runStart < currentRunFrames[runIndex] - 1) {
-            /**
-             * not the last frame for this run
-             */
-            existNext = true;
-            if (tupleIndices[runIndex] >= tupleAccessors[currentFrameIndexInRun[runIndex]].getTupleCount()) {
-                tupleIndices[runIndex] = 0;
-                currentFrameIndexInRun[runIndex]++;
-            }
-        } else if (tupleIndices[runIndex] < tupleAccessors[currentFrameIndexInRun[runIndex]].getTupleCount()) {
-            /**
-             * the last frame has expired
-             */
-            existNext = true;
-        } else {
-            /**
-             * If all tuples in the targeting frame have been
-             * checked.
-             */
-            tupleIndices[runIndex] = 0;
-            currentFrameIndexInRun[runIndex] = runStart;
-            /**
-             * read in batch
-             */
-            currentRunFrames[runIndex] = 0;
-            for (int j = 0; j < runFrameLimit; j++) {
-                int frameIndex = currentFrameIndexInRun[runIndex] + j;
-                if (runCursors[runIndex].nextFrame(inFrames.get(frameIndex))) {
-                    tupleAccessors[frameIndex].reset(inFrames.get(frameIndex).getBuffer());
-                    existNext = true;
-                    currentRunFrames[runIndex]++;
-                } else {
-                    break;
-                }
-            }
-        }
-
-        if (existNext) {
-            topTuples.popAndReplace(tupleAccessors[currentFrameIndexInRun[runIndex]], tupleIndices[runIndex]);
-        } else {
-            topTuples.pop();
-            closeRun(runIndex, runCursors, tupleAccessors);
-        }
-    }
-
-    /**
-     * Close the run file, and also the corresponding readers and
-     * input frame.
-     *
-     * @param index
-     * @param runCursors
-     * @param tupleAccessor
-     * @throws HyracksDataException
-     */
-    private void closeRun(int index, RunFileReader[] runCursors, IFrameTupleAccessor[] tupleAccessor)
-            throws HyracksDataException {
-        if (runCursors[index] != null) {
-            runCursors[index].close();
-            runCursors[index] = null;
-            int frameOffset = index * runFrameLimit;
-            for (int j = 0; j < runFrameLimit; j++) {
-                tupleAccessor[frameOffset + j] = null;
-            }
-        }
-    }
-
-    private int compareFrameTuples(IFrameTupleAccessor fta1, int j1, IFrameTupleAccessor fta2, int j2)
-            throws HyracksDataException {
-        byte[] b1 = fta1.getBuffer().array();
-        byte[] b2 = fta2.getBuffer().array();
-        for (int f = 0; f < keyFields.length; ++f) {
-            int fIdx = f;
-            int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength() + fta1.getFieldStartOffset(j1, fIdx);
-            int l1 = fta1.getFieldLength(j1, fIdx);
-            int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength() + fta2.getFieldStartOffset(j2, fIdx);
-            int l2_start = fta2.getFieldStartOffset(j2, fIdx);
-            int l2_end = fta2.getFieldEndOffset(j2, fIdx);
-            int l2 = l2_end - l2_start;
-            int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
-            if (c != 0) {
-                return c;
-            }
-        }
-        return 0;
-    }
-
-    private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators)
-            throws HyracksDataException {
-        return new Comparator<ReferenceEntry>() {
-
-            @Override
-            public int compare(ReferenceEntry o1, ReferenceEntry o2) {
-                FrameTupleAccessor fta1 = (FrameTupleAccessor) o1.getAccessor();
-                FrameTupleAccessor fta2 = (FrameTupleAccessor) o2.getAccessor();
-                int j1 = o1.getTupleIndex();
-                int j2 = o2.getTupleIndex();
-                byte[] b1 = fta1.getBuffer().array();
-                byte[] b2 = fta2.getBuffer().array();
-                for (int f = 0; f < keyFields.length; ++f) {
-                    int fIdx = 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;
-                    try {
-                        c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
-                        if (c != 0) {
-                            return c;
-                        }
-                    } catch (HyracksDataException e) {
-                        throw new IllegalArgumentException(e);
-                    }
-
-                }
-                return 0;
-            }
-
-        };
-    }
-}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
index a0fd5f8..433b75d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
@@ -48,17 +48,21 @@
     private final IBinaryComparatorFactory[] comparatorFactories;
     private final INormalizedKeyComputerFactory firstNormalizerFactory;
 
-    private final IAggregatorDescriptorFactory aggregatorFactory;
-    private final IAggregatorDescriptorFactory mergerFactory;
+    private final IAggregatorDescriptorFactory partialAggregatorFactory;
+    private final IAggregatorDescriptorFactory intermediateAggregateFactory;
 
     private final int framesLimit;
     private final ISpillableTableFactory spillableTableFactory;
-    private final boolean isOutputSorted;
+    private final RecordDescriptor partialRecDesc;
+    private final RecordDescriptor outRecDesc;
+    private final int tableSize;
+    private final long fileSize;
 
-    public ExternalGroupOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keyFields, int framesLimit,
-            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory firstNormalizerFactory,
-            IAggregatorDescriptorFactory aggregatorFactory, IAggregatorDescriptorFactory mergerFactory,
-            RecordDescriptor recordDescriptor, ISpillableTableFactory spillableTableFactory, boolean isOutputSorted) {
+    public ExternalGroupOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputSizeInTuple, long inputFileSize,
+            int[] keyFields, int framesLimit, IBinaryComparatorFactory[] comparatorFactories,
+            INormalizedKeyComputerFactory firstNormalizerFactory, IAggregatorDescriptorFactory partialAggregatorFactory,
+            IAggregatorDescriptorFactory intermediateAggregateFactory, RecordDescriptor partialAggRecordDesc,
+            RecordDescriptor outRecordDesc, ISpillableTableFactory spillableTableFactory) {
         super(spec, 1, 1);
         this.framesLimit = framesLimit;
         if (framesLimit <= 1) {
@@ -68,19 +72,23 @@
              */
             throw new IllegalStateException("frame limit should at least be 2, but it is " + framesLimit + "!");
         }
-        this.aggregatorFactory = aggregatorFactory;
-        this.mergerFactory = mergerFactory;
+        this.partialAggregatorFactory = partialAggregatorFactory;
+        this.intermediateAggregateFactory = intermediateAggregateFactory;
         this.keyFields = keyFields;
         this.comparatorFactories = comparatorFactories;
         this.firstNormalizerFactory = firstNormalizerFactory;
         this.spillableTableFactory = spillableTableFactory;
-        this.isOutputSorted = isOutputSorted;
+
+        this.partialRecDesc = partialAggRecordDesc;
+        this.outRecDesc = outRecordDesc;
 
         /**
          * Set the record descriptor. Note that since this operator is a unary
          * operator, only the first record descriptor is used here.
          */
-        recordDescriptors[0] = recordDescriptor;
+        recordDescriptors[0] = outRecordDesc;
+        this.tableSize = inputSizeInTuple;
+        this.fileSize = inputFileSize;
     }
 
     /*
@@ -114,11 +122,11 @@
         @Override
         public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
                 final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
-                throws HyracksDataException {
-            return new ExternalGroupBuildOperatorNodePushable(ctx, new TaskId(getActivityId(), partition), keyFields,
-                    framesLimit, comparatorFactories, firstNormalizerFactory, aggregatorFactory,
-                    recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), recordDescriptors[0],
-                    spillableTableFactory);
+                        throws HyracksDataException {
+            return new ExternalGroupBuildOperatorNodePushable(ctx, new TaskId(getActivityId(), partition), tableSize,
+                    fileSize, keyFields, framesLimit, comparatorFactories, firstNormalizerFactory,
+                    partialAggregatorFactory, recordDescProvider.getInputRecordDescriptor(getActivityId(), 0),
+                    recordDescriptors[0], spillableTableFactory);
         }
     }
 
@@ -132,10 +140,12 @@
         @Override
         public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
                 IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
-                throws HyracksDataException {
-            return new ExternalGroupMergeOperatorNodePushable(ctx, new TaskId(new ActivityId(getOperatorId(),
-                    AGGREGATE_ACTIVITY_ID), partition), comparatorFactories, firstNormalizerFactory, keyFields,
-                    mergerFactory, isOutputSorted, framesLimit, recordDescriptors[0]);
+                        throws HyracksDataException {
+            return new ExternalGroupWriteOperatorNodePushable(ctx,
+                    new TaskId(new ActivityId(getOperatorId(), AGGREGATE_ACTIVITY_ID), partition),
+                    spillableTableFactory, partialRecDesc, outRecDesc, framesLimit, keyFields, firstNormalizerFactory,
+                    comparatorFactories, intermediateAggregateFactory);
+
         }
 
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupState.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupState.java
index ebc3516..a36e38d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupState.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupState.java
@@ -21,30 +21,27 @@
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.LinkedList;
 
 import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.dataflow.common.io.RunFileReader;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
 import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
 import org.apache.hyracks.dataflow.std.group.ISpillableTable;
 
 public class ExternalGroupState extends AbstractStateObject {
-    private LinkedList<RunFileReader> runs;
 
+    private RunFileWriter[] runs;
     private ISpillableTable gTable;
-
-    public ExternalGroupState() {
-    }
+    private int[] spilledNumTuples;
 
     ExternalGroupState(JobId jobId, Object id) {
         super(jobId, id);
     }
 
-    public LinkedList<RunFileReader> getRuns() {
+    public RunFileWriter[] getRuns() {
         return runs;
     }
 
-    public void setRuns(LinkedList<RunFileReader> runs) {
+    public void setRuns(RunFileWriter[] runs) {
         this.runs = runs;
     }
 
@@ -65,4 +62,12 @@
     public void fromBytes(DataInput in) throws IOException {
         throw new UnsupportedOperationException();
     }
+
+    public void setSpilledNumTuples(int[] spilledNumTuples) {
+        this.spilledNumTuples = spilledNumTuples;
+    }
+
+    public int[] getSpilledNumTuples() {
+        return spilledNumTuples;
+    }
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupWriteOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupWriteOperatorNodePushable.java
new file mode 100644
index 0000000..a2922ae
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalGroupWriteOperatorNodePushable.java
@@ -0,0 +1,167 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.group.external;
+
+import java.util.ArrayList;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.dataflow.common.io.RunFileReader;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.group.AggregateType;
+import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.ISpillableTable;
+import org.apache.hyracks.dataflow.std.group.ISpillableTableFactory;
+
+public class ExternalGroupWriteOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable
+        implements IRunFileWriterGenerator {
+    private static Logger LOGGER = Logger.getLogger("ExternalGroupbyWrite");
+    private final IHyracksTaskContext ctx;
+    private final Object stateId;
+    private final ISpillableTableFactory spillableTableFactory;
+    private final RecordDescriptor partialAggRecordDesc;
+    private final RecordDescriptor outRecordDesc;
+    private final IAggregatorDescriptorFactory mergeAggregatorFactory;
+    private final int[] mergeGroupFields;
+    private final IBinaryComparator[] groupByComparators;
+    private final int frameLimit;
+    private final INormalizedKeyComputer nmkComputer;
+    private final ArrayList<RunFileWriter> generatedRuns = new ArrayList<>();
+
+    public ExternalGroupWriteOperatorNodePushable(IHyracksTaskContext ctx, Object stateId,
+            ISpillableTableFactory spillableTableFactory, RecordDescriptor partialAggRecordDesc,
+            RecordDescriptor outRecordDesc, int framesLimit, int[] groupFields,
+            INormalizedKeyComputerFactory nmkFactory, IBinaryComparatorFactory[] comparatorFactories,
+            IAggregatorDescriptorFactory aggregatorFactory) {
+        this.ctx = ctx;
+        this.stateId = stateId;
+        this.spillableTableFactory = spillableTableFactory;
+        this.frameLimit = framesLimit;
+        this.nmkComputer = nmkFactory.createNormalizedKeyComputer();
+
+        this.partialAggRecordDesc = partialAggRecordDesc;
+        this.outRecordDesc = outRecordDesc;
+
+        this.mergeAggregatorFactory = aggregatorFactory;
+
+        //create merge group fields
+        int numGroupFields = groupFields.length;
+        mergeGroupFields = new int[numGroupFields];
+        for (int i = 0; i < numGroupFields; i++) {
+            mergeGroupFields[i] = i;
+        }
+
+        //setup comparators for grouping
+        groupByComparators = new IBinaryComparator[Math.min(mergeGroupFields.length, comparatorFactories.length)];
+        for (int i = 0; i < groupByComparators.length; i++) {
+            groupByComparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+        ExternalGroupState aggState = (ExternalGroupState) ctx.getStateObject(stateId);
+        ISpillableTable table = aggState.getSpillableTable();
+        RunFileWriter[] partitionRuns = aggState.getRuns();
+        int[] numberOfTuples = aggState.getSpilledNumTuples();
+        try {
+            writer.open();
+            doPass(table, partitionRuns, numberOfTuples, writer, 1); // level 0 use used at build stage.
+        } catch (Exception e) {
+            generatedRuns.forEach(run -> run.getFileReference().delete());
+            writer.fail();
+            throw new HyracksDataException(e);
+        } finally {
+            writer.close();
+        }
+    }
+
+    private void doPass(ISpillableTable table, RunFileWriter[] runs, int[] numOfTuples, IFrameWriter writer, int level)
+            throws HyracksDataException {
+        assert table.getNumPartitions() == runs.length;
+        for (int i = 0; i < runs.length; i++) {
+            if (runs[i] == null) {
+                table.flushFrames(i, writer, AggregateType.FINAL);
+            }
+        }
+        table.close();
+
+        for (int i = 0; i < runs.length; i++) {
+            if (runs[i] != null) {
+                ISpillableTable partitionTable = spillableTableFactory.buildSpillableTable(ctx, numOfTuples[i],
+                        runs[i].getFileSize(), mergeGroupFields, groupByComparators, nmkComputer,
+                        mergeAggregatorFactory, partialAggRecordDesc, outRecordDesc, frameLimit, level);
+                RunFileWriter[] runFileWriters = new RunFileWriter[partitionTable.getNumPartitions()];
+                int[] sizeInTuplesNextLevel = buildGroup(runs[i].createDeleteOnCloseReader(), partitionTable,
+                        runFileWriters);
+                for (int idFile = 0; idFile < runFileWriters.length; idFile++) {
+                    if (runFileWriters[idFile] != null) {
+                        generatedRuns.add(runFileWriters[idFile]);
+                    }
+                }
+
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    int numOfSpilledPart = 0;
+                    for (int x = 0; x < numOfTuples.length; x++) {
+                        if (numOfTuples[x] > 0) {
+                            numOfSpilledPart++;
+                        }
+                    }
+                    LOGGER.fine("level " + level + ":" + "build with " + numOfTuples.length + " partitions"
+                            + ", spilled " + numOfSpilledPart + " partitions");
+                }
+                doPass(partitionTable, runFileWriters, sizeInTuplesNextLevel, writer, level + 1);
+            }
+        }
+    }
+
+    private int[] buildGroup(RunFileReader reader, ISpillableTable table, RunFileWriter[] runFileWriters)
+            throws HyracksDataException {
+        ExternalHashGroupBy groupBy = new ExternalHashGroupBy(this, table, runFileWriters, partialAggRecordDesc);
+        reader.open();
+        try {
+            VSizeFrame frame = new VSizeFrame(ctx);
+            while (reader.nextFrame(frame)) {
+                groupBy.insert(frame.getBuffer());
+            }
+            groupBy.flushSpilledPartitions();
+        } finally {
+            reader.close();
+        }
+        return groupBy.getSpilledNumTuples();
+    }
+
+    @Override
+    public RunFileWriter getRunFileWriter() throws HyracksDataException {
+        FileReference newRun = ctx.getJobletContext()
+                .createManagedWorkspaceFile(ExternalGroupOperatorDescriptor.class.getSimpleName());
+        return new RunFileWriter(newRun, ctx.getIOManager());
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalHashGroupBy.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalHashGroupBy.java
new file mode 100644
index 0000000..e0ef2b3
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/ExternalHashGroupBy.java
@@ -0,0 +1,97 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.group.external;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.group.AggregateType;
+import org.apache.hyracks.dataflow.std.group.ISpillableTable;
+
+public class ExternalHashGroupBy {
+
+    private final IRunFileWriterGenerator runFileWriterGenerator;
+
+    private FrameTupleAccessor accessor;
+    private ISpillableTable table;
+    private RunFileWriter[] runWriters;
+    private int[] spilledNumTuples;
+
+    public ExternalHashGroupBy(IRunFileWriterGenerator runFileWriterGenerator, ISpillableTable table,
+            RunFileWriter[] runWriters, RecordDescriptor inRecordDescriptor) {
+        this.runFileWriterGenerator = runFileWriterGenerator;
+        this.table = table;
+        this.runWriters = runWriters;
+        this.accessor = new FrameTupleAccessor(inRecordDescriptor);
+        this.spilledNumTuples = new int[runWriters.length];
+    }
+
+    public void insert(ByteBuffer buffer) throws HyracksDataException {
+        accessor.reset(buffer);
+        int tupleCount = accessor.getTupleCount();
+        for (int i = 0; i < tupleCount; i++) {
+            if (!table.insert(accessor, i)) {
+                do {
+                    int partition = table.findVictimPartition(accessor, i);
+                    if (partition < 0) {
+                        throw new HyracksDataException("Failed to insert a new buffer into the aggregate operator!");
+                    }
+                    RunFileWriter writer = getPartitionWriterOrCreateOneIfNotExist(partition);
+                    flushPartitionToRun(partition, writer);
+                } while (!table.insert(accessor, i));
+            }
+        }
+    }
+
+    private void flushPartitionToRun(int partition, RunFileWriter writer)
+            throws HyracksDataException {
+        try {
+            spilledNumTuples[partition] += table.flushFrames(partition, writer, AggregateType.PARTIAL);
+            table.clear(partition);
+        } catch (Exception ex) {
+            writer.fail();
+            throw new HyracksDataException(ex);
+        }
+    }
+
+    public void flushSpilledPartitions() throws HyracksDataException {
+        for (int i = 0; i < runWriters.length; ++i) {
+            if (runWriters[i] != null) {
+                flushPartitionToRun(i, runWriters[i]);
+                runWriters[i].close();
+            }
+        }
+    }
+
+    private RunFileWriter getPartitionWriterOrCreateOneIfNotExist(int partition) throws HyracksDataException {
+        if (runWriters[partition] == null) {
+            runWriters[partition] = runFileWriterGenerator.getRunFileWriter();
+            runWriters[partition].open();
+        }
+        return runWriters[partition];
+    }
+
+    public int[] getSpilledNumTuples() {
+        return spilledNumTuples;
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/IRunFileWriterGenerator.java
similarity index 73%
copy from hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
copy to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/IRunFileWriterGenerator.java
index 21ed3e2..58c6ae1 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/external/IRunFileWriterGenerator.java
@@ -17,10 +17,11 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.group.external;
 
-public enum EnumFreeSlotPolicy {
-    SMALLEST_FIT,
-    LAST_FIT,
-    BIGGEST_FIT,
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+
+public interface IRunFileWriterGenerator {
+    RunFileWriter getRunFileWriter() throws HyracksDataException;
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
deleted file mode 100644
index 2f4a0b2..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/*
- * 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 org.apache.hyracks.dataflow.std.group.hash;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.hyracks.api.comm.IFrame;
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
-import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
-import org.apache.hyracks.dataflow.std.group.AggregateState;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptor;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
-
-class GroupingHashTable {
-    /**
-     * The pointers in the link store 3 int values for each entry in the
-     * hashtable: (bufferIdx, tIndex, accumulatorIdx).
-     *
-     * @author vinayakb
-     */
-    private static class Link {
-        private static final int INIT_POINTERS_SIZE = 9;
-
-        int[] pointers;
-        int size;
-
-        Link() {
-            pointers = new int[INIT_POINTERS_SIZE];
-            size = 0;
-        }
-
-        void add(int bufferIdx, int tIndex, int accumulatorIdx) {
-            while (size + 3 > pointers.length) {
-                pointers = Arrays.copyOf(pointers, pointers.length * 2);
-            }
-            pointers[size++] = bufferIdx;
-            pointers[size++] = tIndex;
-            pointers[size++] = accumulatorIdx;
-        }
-    }
-
-    private static final int INIT_AGG_STATE_SIZE = 8;
-    private final IHyracksTaskContext ctx;
-
-    private final List<IFrame> buffers;
-    private final Link[] table;
-    /**
-     * Aggregate states: a list of states for all groups maintained in the main
-     * memory.
-     */
-    private AggregateState[] aggregateStates;
-    private int accumulatorSize;
-
-    private int lastBIndex;
-    private final int[] storedKeys;
-    private final int[] keys;
-    private final IBinaryComparator[] comparators;
-    private final FrameTuplePairComparator ftpc;
-    private final ITuplePartitionComputer tpc;
-    private final IAggregatorDescriptor aggregator;
-
-    private final IFrame outputFrame;
-    private final FrameTupleAppender appender;
-
-    private final FrameTupleAccessor storedKeysAccessor;
-
-    private final ArrayTupleBuilder stateTupleBuilder, outputTupleBuilder;
-
-    GroupingHashTable(IHyracksTaskContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
-            ITuplePartitionComputerFactory tpcf, IAggregatorDescriptorFactory aggregatorFactory,
-            RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int tableSize)
-            throws HyracksDataException {
-        this.ctx = ctx;
-
-        buffers = new ArrayList<>();
-        table = new Link[tableSize];
-
-        keys = fields;
-        storedKeys = new int[fields.length];
-        @SuppressWarnings("rawtypes")
-        ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[fields.length];
-        for (int i = 0; i < fields.length; ++i) {
-            storedKeys[i] = i;
-            storedKeySerDeser[i] = inRecordDescriptor.getFields()[fields[i]];
-        }
-
-        comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
-        ftpc = new FrameTuplePairComparator(fields, storedKeys, comparators);
-        tpc = tpcf.createPartitioner();
-
-        int[] keyFieldsInPartialResults = new int[fields.length];
-        for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
-            keyFieldsInPartialResults[i] = i;
-        }
-
-        this.aggregator = aggregatorFactory.createAggregator(ctx, inRecordDescriptor, outRecordDescriptor, fields,
-                keyFieldsInPartialResults, null);
-
-        this.aggregateStates = new AggregateState[INIT_AGG_STATE_SIZE];
-        accumulatorSize = 0;
-
-        RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(storedKeySerDeser);
-        storedKeysAccessor = new FrameTupleAccessor(storedKeysRecordDescriptor);
-        lastBIndex = -1;
-
-        appender = new FrameTupleAppender();
-
-        addNewBuffer();
-
-        if (fields.length < outRecordDescriptor.getFields().length) {
-            stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
-        } else {
-            stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length + 1);
-        }
-        outputTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
-        outputFrame = new VSizeFrame(ctx);
-    }
-
-    private void addNewBuffer() throws HyracksDataException {
-        VSizeFrame frame = new VSizeFrame(ctx);
-        buffers.add(frame);
-        appender.reset(frame, true);
-        ++lastBIndex;
-    }
-
-    void insert(FrameTupleAccessor accessor, int tIndex) throws Exception {
-        int entry = tpc.partition(accessor, tIndex, table.length);
-        Link link = table[entry];
-        if (link == null) {
-            link = table[entry] = new Link();
-        }
-        int saIndex = -1;
-        for (int i = 0; i < link.size; i += 3) {
-            int sbIndex = link.pointers[i];
-            int stIndex = link.pointers[i + 1];
-            storedKeysAccessor.reset(buffers.get(sbIndex).getBuffer());
-            int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
-            if (c == 0) {
-                saIndex = link.pointers[i + 2];
-                break;
-            }
-        }
-        if (saIndex < 0) {
-            // Did not find the key. Insert a new entry.
-            saIndex = accumulatorSize++;
-            // Add keys
-
-            // Add aggregation fields
-            AggregateState newState = aggregator.createAggregateStates();
-
-            stateTupleBuilder.reset();
-            for (int k = 0; k < keys.length; k++) {
-                stateTupleBuilder.addField(accessor, tIndex, keys[k]);
-            }
-
-            aggregator.init(stateTupleBuilder, accessor, tIndex, newState);
-
-            if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
-                    stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
-                addNewBuffer();
-                if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
-                        stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
-                    throw new HyracksDataException("Cannot init the aggregate state in a single frame.");
-                }
-            }
-
-            if (accumulatorSize >= aggregateStates.length) {
-                aggregateStates = Arrays.copyOf(aggregateStates, aggregateStates.length * 2);
-            }
-
-            aggregateStates[saIndex] = newState;
-
-            link.add(lastBIndex, appender.getTupleCount() - 1, saIndex);
-
-        } else {
-            aggregator.aggregate(accessor, tIndex, null, 0, aggregateStates[saIndex]);
-        }
-    }
-
-    void write(IFrameWriter writer) throws HyracksDataException {
-        appender.reset(outputFrame, true);
-
-        for (int i = 0; i < table.length; ++i) {
-            Link link = table[i];
-            if (link != null) {
-                for (int j = 0; j < link.size; j += 3) {
-                    int bIndex = link.pointers[j];
-                    int tIndex = link.pointers[j + 1];
-                    int aIndex = link.pointers[j + 2];
-                    ByteBuffer keyBuffer = buffers.get(bIndex).getBuffer();
-                    storedKeysAccessor.reset(keyBuffer);
-
-                    // copy keys
-                    outputTupleBuilder.reset();
-                    for (int k = 0; k < storedKeys.length; k++) {
-                        outputTupleBuilder.addField(storedKeysAccessor, tIndex, storedKeys[k]);
-                    }
-
-                    aggregator.outputFinalResult(outputTupleBuilder, storedKeysAccessor, tIndex,
-                            aggregateStates[aIndex]);
-
-                    FrameUtils.appendSkipEmptyFieldToWriter(writer, appender, outputTupleBuilder.getFieldEndOffsets(),
-                            outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize());
-
-                }
-            }
-        }
-        appender.write(writer, true);
-    }
-
-    void close() throws HyracksDataException {
-        for (AggregateState aState : aggregateStates) {
-            aState.close();
-        }
-    }
-}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java
deleted file mode 100644
index 902021f..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * 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 org.apache.hyracks.dataflow.std.group.hash;
-
-import java.nio.ByteBuffer;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
-
-class HashGroupBuildOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
-    private final IHyracksTaskContext ctx;
-    private final FrameTupleAccessor accessor;
-    private final Object stateId;
-    private final int[] keys;
-    private final ITuplePartitionComputerFactory tpcf;
-    private final IBinaryComparatorFactory[] comparatorFactories;
-    private final IAggregatorDescriptorFactory aggregatorFactory;
-    private final int tableSize;
-    private final RecordDescriptor inRecordDescriptor;
-    private final RecordDescriptor outRecordDescriptor;
-
-    private HashGroupState state;
-
-    HashGroupBuildOperatorNodePushable(IHyracksTaskContext ctx, Object stateId, int[] keys,
-            ITuplePartitionComputerFactory tpcf, IBinaryComparatorFactory[] comparatorFactories,
-            IAggregatorDescriptorFactory aggregatorFactory, int tableSize, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor) {
-        this.ctx = ctx;
-        this.accessor = new FrameTupleAccessor(inRecordDescriptor);
-        this.stateId = stateId;
-        this.keys = keys;
-        this.tpcf = tpcf;
-        this.comparatorFactories = comparatorFactories;
-        this.aggregatorFactory = aggregatorFactory;
-        this.tableSize = tableSize;
-        this.inRecordDescriptor = inRecordDescriptor;
-        this.outRecordDescriptor = outRecordDescriptor;
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        state = new HashGroupState(ctx.getJobletContext().getJobId(), stateId);
-        state.setHashTable(new GroupingHashTable(ctx, keys, comparatorFactories, tpcf, aggregatorFactory,
-                inRecordDescriptor, outRecordDescriptor, tableSize));
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        GroupingHashTable table = state.getHashTable();
-        accessor.reset(buffer);
-        int tupleCount = accessor.getTupleCount();
-        for (int i = 0; i < tupleCount; ++i) {
-            try {
-                table.insert(accessor, i);
-            } catch (Exception e) {
-                System.out.println(e.toString());
-                throw new HyracksDataException(e);
-            }
-        }
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        ctx.setStateObject(state);
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        throw new HyracksDataException("HashGroupOperator is failed.");
-    }
-}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupOperatorDescriptor.java
deleted file mode 100644
index a0bbb2d..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupOperatorDescriptor.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * 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 org.apache.hyracks.dataflow.std.group.hash;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.ActivityId;
-import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
-import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
-import org.apache.hyracks.api.dataflow.TaskId;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
-import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
-
-/**
- *
- */
-public class HashGroupOperatorDescriptor extends AbstractOperatorDescriptor {
-
-    private static final int HASH_BUILD_ACTIVITY_ID = 0;
-
-    private static final int OUTPUT_ACTIVITY_ID = 1;
-
-    private static final long serialVersionUID = 1L;
-
-    private final int[] keys;
-    private final ITuplePartitionComputerFactory tpcf;
-    private final IBinaryComparatorFactory[] comparatorFactories;
-
-    private final IAggregatorDescriptorFactory aggregatorFactory;
-
-    private final int tableSize;
-
-    public HashGroupOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys,
-            ITuplePartitionComputerFactory tpcf, IBinaryComparatorFactory[] comparatorFactories,
-            IAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor outRecordDescriptor, int tableSize) {
-        super(spec, 1, 1);
-        this.keys = keys;
-        this.tpcf = tpcf;
-        this.comparatorFactories = comparatorFactories;
-        this.aggregatorFactory = aggregatorFactory;
-        recordDescriptors[0] = outRecordDescriptor;
-        this.tableSize = tableSize;
-    }
-
-    /*
-     * (non-Javadoc)
-     *
-     * @see
-     * org.apache.hyracks.api.dataflow.IOperatorDescriptor#contributeActivities
-     * (org.apache.hyracks.api.dataflow.IActivityGraphBuilder)
-     */
-    @Override
-    public void contributeActivities(IActivityGraphBuilder builder) {
-        HashBuildActivity ha = new HashBuildActivity(new ActivityId(odId, HASH_BUILD_ACTIVITY_ID));
-        builder.addActivity(this, ha);
-
-        OutputActivity oa = new OutputActivity(new ActivityId(odId, OUTPUT_ACTIVITY_ID));
-        builder.addActivity(this, oa);
-
-        builder.addSourceEdge(0, ha, 0);
-        builder.addTargetEdge(0, oa, 0);
-        builder.addBlockingEdge(ha, oa);
-    }
-
-    private class HashBuildActivity extends AbstractActivityNode {
-        private static final long serialVersionUID = 1L;
-
-        public HashBuildActivity(ActivityId id) {
-            super(id);
-        }
-
-        @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
-            return new HashGroupBuildOperatorNodePushable(ctx, new TaskId(getActivityId(), partition), keys, tpcf,
-                    comparatorFactories, aggregatorFactory, tableSize, recordDescProvider.getInputRecordDescriptor(
-                            getActivityId(), 0), recordDescriptors[0]);
-        }
-    }
-
-    private class OutputActivity extends AbstractActivityNode {
-        private static final long serialVersionUID = 1L;
-
-        public OutputActivity(ActivityId id) {
-            super(id);
-        }
-
-        @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
-            return new HashGroupOutputOperatorNodePushable(ctx, new TaskId(new ActivityId(getOperatorId(),
-                    HASH_BUILD_ACTIVITY_ID), partition));
-        }
-    }
-}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupOutputOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupOutputOperatorNodePushable.java
deleted file mode 100644
index ce0ce34..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupOutputOperatorNodePushable.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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 org.apache.hyracks.dataflow.std.group.hash;
-
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-
-class HashGroupOutputOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
-    private final IHyracksTaskContext ctx;
-    private final Object stateId;
-
-    HashGroupOutputOperatorNodePushable(IHyracksTaskContext ctx, Object stateId) {
-        this.ctx = ctx;
-        this.stateId = stateId;
-    }
-
-    @Override
-    public void initialize() throws HyracksDataException {
-        HashGroupState buildState = (HashGroupState) ctx.getStateObject(stateId);
-        GroupingHashTable table = buildState.getHashTable();
-        writer.open();
-        try {
-            table.write(writer);
-        } catch (Exception e) {
-            writer.fail();
-            throw new HyracksDataException(e);
-        } finally {
-            writer.close();
-        }
-    }
-}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupState.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupState.java
deleted file mode 100644
index b5cf274..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/HashGroupState.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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 org.apache.hyracks.dataflow.std.group.hash;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
-
-public class HashGroupState extends AbstractStateObject {
-    private GroupingHashTable table;
-
-    public HashGroupState() {
-    }
-
-    HashGroupState(JobId jobId, Object id) {
-        super(jobId, id);
-    }
-
-    public GroupingHashTable getHashTable() {
-        return table;
-    }
-
-    public void setHashTable(GroupingHashTable table) {
-        this.table = table;
-    }
-
-    @Override
-    public void toBytes(DataOutput out) throws IOException {
-
-    }
-
-    @Override
-    public void fromBytes(DataInput in) throws IOException {
-
-    }
-}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
index a17839a..091d323 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
@@ -27,18 +27,18 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
 import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupWriter;
+import org.apache.hyracks.dataflow.std.sort.AbstractExternalSortRunGenerator;
 import org.apache.hyracks.dataflow.std.sort.Algorithm;
-import org.apache.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
 
 /**
  * Group-by aggregation is pushed before run file generation.
  *
  * @author yingyib
  */
-public class ExternalSortGroupByRunGenerator extends ExternalSortRunGenerator {
+public class ExternalSortGroupByRunGenerator extends AbstractExternalSortRunGenerator {
 
     private final int[] groupFields;
     private final IBinaryComparatorFactory[] comparatorFactories;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
index 7f23bd9..b13c647 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
@@ -27,19 +27,19 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
 import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupWriter;
-import org.apache.hyracks.dataflow.std.sort.ExternalSortRunMerger;
+import org.apache.hyracks.dataflow.std.sort.AbstractExternalSortRunMerger;
 import org.apache.hyracks.dataflow.std.sort.ISorter;
-import org.apache.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
 
 /**
  * Group-by aggregation is pushed into multi-pass merge of external sort.
  *
  * @author yingyib
  */
-public class ExternalSortGroupByRunMerger extends ExternalSortRunMerger {
+public class ExternalSortGroupByRunMerger extends AbstractExternalSortRunMerger {
 
     private final RecordDescriptor inputRecordDesc;
     private final RecordDescriptor partialAggRecordDesc;
@@ -54,14 +54,14 @@
     private final int[] mergeGroupFields;
     private final IBinaryComparator[] groupByComparators;
 
-    public ExternalSortGroupByRunMerger(IHyracksTaskContext ctx, ISorter frameSorter, List<RunAndMaxFrameSizePair> runs,
+    public ExternalSortGroupByRunMerger(IHyracksTaskContext ctx, ISorter frameSorter, List<GeneratedRunFileReader> runs,
             int[] sortFields, RecordDescriptor inRecordDesc, RecordDescriptor partialAggRecordDesc,
             RecordDescriptor outRecordDesc, int framesLimit, IFrameWriter writer, int[] groupFields,
             INormalizedKeyComputer nmk, IBinaryComparator[] comparators,
             IAggregatorDescriptorFactory partialAggregatorFactory, IAggregatorDescriptorFactory aggregatorFactory,
             boolean localStage) {
-        super(ctx, frameSorter, runs, sortFields, comparators, nmk, partialAggRecordDesc, framesLimit,
-                writer);
+        super(ctx, frameSorter, runs, comparators, nmk, partialAggRecordDesc, framesLimit, writer);
+
         this.inputRecordDesc = inRecordDesc;
         this.partialAggRecordDesc = partialAggRecordDesc;
         this.outRecordDesc = outRecordDesc;
@@ -96,8 +96,8 @@
     protected IFrameWriter prepareSkipMergingFinalResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
         IAggregatorDescriptorFactory aggregatorFactory = localSide ? partialAggregatorFactory : mergeAggregatorFactory;
         boolean outputPartial = false;
-        return new PreclusteredGroupWriter(ctx, groupFields, groupByComparators,
-                aggregatorFactory, inputRecordDesc, outRecordDesc, nextWriter, outputPartial);
+        return new PreclusteredGroupWriter(ctx, groupFields, groupByComparators, aggregatorFactory, inputRecordDesc,
+                outRecordDesc, nextWriter, outputPartial);
     }
 
     @Override
@@ -118,8 +118,8 @@
     @Override
     protected IFrameWriter prepareFinalMergeResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
         boolean outputPartial = false;
-        return new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators,
-                mergeAggregatorFactory, partialAggRecordDesc, outRecordDesc, nextWriter, outputPartial);
+        return new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators, mergeAggregatorFactory,
+                partialAggRecordDesc, outRecordDesc, nextWriter, outputPartial);
     }
 
     @Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
index 0af47b9..16d2158 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
@@ -31,13 +31,13 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
 import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.sort.AbstractExternalSortRunMerger;
 import org.apache.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
 import org.apache.hyracks.dataflow.std.sort.AbstractSorterOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.Algorithm;
-import org.apache.hyracks.dataflow.std.sort.ExternalSortRunMerger;
 import org.apache.hyracks.dataflow.std.sort.ISorter;
-import org.apache.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
 
 /**
  * This Operator pushes group-by aggregation into the external sort.
@@ -57,17 +57,28 @@
     private Algorithm alg = Algorithm.MERGE_SORT;
 
     /**
-     * @param spec                      , the Hyracks job specification
-     * @param framesLimit               , the frame limit for this operator
-     * @param sortFields                , the fields to sort
-     * @param groupFields               , the fields to group, which can be a prefix subset of sortFields
-     * @param firstKeyNormalizerFactory , the normalized key computer factory of the first key
-     * @param comparatorFactories       , the comparator factories of sort keys
-     * @param partialAggregatorFactory  , for aggregating the input of this operator
-     * @param mergeAggregatorFactory    , for aggregating the intermediate data of this operator
-     * @param partialAggRecordDesc      , the record descriptor of intermediate data
-     * @param outRecordDesc             , the record descriptor of output data
-     * @param finalStage                , whether the operator is used for final stage aggregation
+     * @param spec
+     *            , the Hyracks job specification
+     * @param framesLimit
+     *            , the frame limit for this operator
+     * @param sortFields
+     *            , the fields to sort
+     * @param groupFields
+     *            , the fields to group, which can be a prefix subset of sortFields
+     * @param firstKeyNormalizerFactory
+     *            , the normalized key computer factory of the first key
+     * @param comparatorFactories
+     *            , the comparator factories of sort keys
+     * @param partialAggregatorFactory
+     *            , for aggregating the input of this operator
+     * @param mergeAggregatorFactory
+     *            , for aggregating the intermediate data of this operator
+     * @param partialAggRecordDesc
+     *            , the record descriptor of intermediate data
+     * @param outRecordDesc
+     *            , the record descriptor of output data
+     * @param finalStage
+     *            , whether the operator is used for final stage aggregation
      */
     public SortGroupByOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
             int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
@@ -88,30 +99,6 @@
         this.finalStage = finalStage;
     }
 
-    /**
-     * @param spec                      , the Hyracks job specification
-     * @param framesLimit               , the frame limit for this operator
-     * @param sortFields                , the fields to sort
-     * @param groupFields               , the fields to group, which can be a prefix subset of sortFields
-     * @param firstKeyNormalizerFactory , the normalized key computer factory of the first key
-     * @param comparatorFactories       , the comparator factories of sort keys
-     * @param partialAggregatorFactory  , for aggregating the input of this operator
-     * @param mergeAggregatorFactory    , for aggregating the intermediate data of this operator
-     * @param partialAggRecordDesc      , the record descriptor of intermediate data
-     * @param outRecordDesc             , the record descriptor of output data
-     * @param finalStage                , whether the operator is used for final stage aggregation
-     * @param alg                       , the in-memory sort algorithm
-     */
-    public SortGroupByOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
-            int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
-            IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory partialAggregatorFactory,
-            IAggregatorDescriptorFactory mergeAggregatorFactory, RecordDescriptor partialAggRecordDesc,
-            RecordDescriptor outRecordDesc, boolean finalStage, Algorithm alg) {
-        this(spec, framesLimit, sortFields, groupFields, firstKeyNormalizerFactory, comparatorFactories,
-                partialAggregatorFactory, mergeAggregatorFactory, partialAggRecordDesc, outRecordDesc, finalStage);
-        this.alg = alg;
-    }
-
     @Override
     public AbstractSorterOperatorDescriptor.SortActivity getSortActivity(ActivityId id) {
         return new AbstractSorterOperatorDescriptor.SortActivity(id) {
@@ -131,9 +118,9 @@
         return new AbstractSorterOperatorDescriptor.MergeActivity(id) {
 
             @Override
-            protected ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
+            protected AbstractExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
                     IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter,
-                    List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators,
+                    List<GeneratedRunFileReader> runs, IBinaryComparator[] comparators,
                     INormalizedKeyComputer nmkComputer, int necessaryFrames) {
                 return new ExternalSortGroupByRunMerger(ctx, sorter, runs, sortFields,
                         recordDescProvider.getInputRecordDescriptor(new ActivityId(odId, SORT_ACTIVITY_ID), 0),
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
index e45f952..69e9e6a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+import org.apache.hyracks.dataflow.std.util.FrameTuplePairComparator;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
 import org.apache.hyracks.dataflow.common.data.partition.RepartitionComputerFactory;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index d215983..d0a81ee 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -47,7 +47,7 @@
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+import org.apache.hyracks.dataflow.std.util.FrameTuplePairComparator;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
 import org.apache.hyracks.dataflow.common.data.partition.RepartitionComputerFactory;
@@ -96,25 +96,6 @@
     public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
             int recordsPerFrame, double factor, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDescriptor, IPredicateEvaluatorFactory predEvalFactory) throws HyracksDataException {
-        super(spec, 2, 1);
-        this.memsize = memsize;
-        this.inputsize0 = inputsize0;
-        this.factor = factor;
-        this.recordsPerFrame = recordsPerFrame;
-        this.keys0 = keys0;
-        this.keys1 = keys1;
-        this.hashFunctionFactories = hashFunctionFactories;
-        this.comparatorFactories = comparatorFactories;
-        this.predEvaluatorFactory = predEvalFactory;
-        this.isLeftOuter = false;
-        this.nullWriterFactories1 = null;
-        recordDescriptors[0] = recordDescriptor;
-    }
-
-    public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
-            int recordsPerFrame, double factor, int[] keys0, int[] keys1,
-            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor, IPredicateEvaluatorFactory predEvalFactory, boolean isLeftOuter,
             INullWriterFactory[] nullWriterFactories1) throws HyracksDataException {
         super(spec, 2, 1);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
index a139341..fee7dd8 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -24,6 +24,7 @@
 import java.util.List;
 import java.util.logging.Logger;
 
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -34,10 +35,10 @@
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.std.structures.ISerializableTable;
 import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+import org.apache.hyracks.dataflow.std.util.FrameTuplePairComparator;
 
 public class InMemoryHashJoin {
 
@@ -45,7 +46,7 @@
     private final List<ByteBuffer> buffers;
     private final FrameTupleAccessor accessorBuild;
     private final ITuplePartitionComputer tpcBuild;
-    private final FrameTupleAccessor accessorProbe;
+    private IFrameTupleAccessor accessorProbe;
     private final ITuplePartitionComputer tpcProbe;
     private final FrameTupleAppender appender;
     private final FrameTuplePairComparator tpComparator;
@@ -59,37 +60,38 @@
 
     private static final Logger LOGGER = Logger.getLogger(InMemoryHashJoin.class.getName());
 
-    public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
-            ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
-            FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1,
+    public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessorProbe,
+            ITuplePartitionComputer tpcProbe, FrameTupleAccessor accessorBuild, ITuplePartitionComputer tpcBuild,
+            FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWritersBuild,
             ISerializableTable table, IPredicateEvaluator predEval) throws HyracksDataException {
-        this(ctx, tableSize, accessor0, tpc0, accessor1, tpc1, comparator, isLeftOuter, nullWriters1, table, predEval,
+        this(ctx, tableSize, accessorProbe, tpcProbe, accessorBuild, tpcBuild, comparator, isLeftOuter,
+                nullWritersBuild, table, predEval,
                 false);
     }
 
-    public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
-            ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
-            FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1,
+    public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessorProbe,
+            ITuplePartitionComputer tpcProbe, FrameTupleAccessor accessorBuild, ITuplePartitionComputer tpcBuild,
+            FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWritersBuild,
             ISerializableTable table, IPredicateEvaluator predEval, boolean reverse) throws HyracksDataException {
         this.ctx = ctx;
         this.tableSize = tableSize;
         this.table = table;
         storedTuplePointer = new TuplePointer();
         buffers = new ArrayList<ByteBuffer>();
-        this.accessorBuild = accessor1;
-        this.tpcBuild = tpc1;
-        this.accessorProbe = accessor0;
-        this.tpcProbe = tpc0;
+        this.accessorBuild = accessorBuild;
+        this.tpcBuild = tpcBuild;
+        this.accessorProbe = accessorProbe;
+        this.tpcProbe = tpcProbe;
         appender = new FrameTupleAppender(new VSizeFrame(ctx));
         tpComparator = comparator;
         predEvaluator = predEval;
         this.isLeftOuter = isLeftOuter;
         if (isLeftOuter) {
-            int fieldCountOuter = accessor1.getFieldCount();
+            int fieldCountOuter = accessorBuild.getFieldCount();
             nullTupleBuild = new ArrayTupleBuilder(fieldCountOuter);
             DataOutput out = nullTupleBuild.getDataOutput();
             for (int i = 0; i < fieldCountOuter; i++) {
-                nullWriters1[i].writeNull(out);
+                nullWritersBuild[i].writeNull(out);
                 nullTupleBuild.addFieldEndOffset();
             }
         } else {
@@ -113,36 +115,41 @@
         }
     }
 
+    void join(IFrameTupleAccessor accessorProbe, int tid, IFrameWriter writer) throws HyracksDataException {
+        this.accessorProbe = accessorProbe;
+        boolean matchFound = false;
+        if (tableSize != 0) {
+            int entry = tpcProbe.partition(accessorProbe, tid, tableSize);
+            int offset = 0;
+            do {
+                table.getTuplePointer(entry, offset++, storedTuplePointer);
+                if (storedTuplePointer.frameIndex < 0)
+                    break;
+                int bIndex = storedTuplePointer.frameIndex;
+                int tIndex = storedTuplePointer.tupleIndex;
+                accessorBuild.reset(buffers.get(bIndex));
+                int c = tpComparator.compare(accessorProbe, tid, accessorBuild, tIndex);
+                if (c == 0) {
+                    boolean predEval = evaluatePredicate(tid, tIndex);
+                    if (predEval) {
+                        matchFound = true;
+                        appendToResult(tid, tIndex, writer);
+                    }
+                }
+            } while (true);
+        }
+        if (!matchFound && isLeftOuter) {
+            FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, tid,
+                    nullTupleBuild.getFieldEndOffsets(), nullTupleBuild.getByteArray(), 0,
+                    nullTupleBuild.getSize());
+        }
+    }
+
     public void join(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
         accessorProbe.reset(buffer);
         int tupleCount0 = accessorProbe.getTupleCount();
         for (int i = 0; i < tupleCount0; ++i) {
-            boolean matchFound = false;
-            if (tableSize != 0) {
-                int entry = tpcProbe.partition(accessorProbe, i, tableSize);
-                int offset = 0;
-                do {
-                    table.getTuplePointer(entry, offset++, storedTuplePointer);
-                    if (storedTuplePointer.frameIndex < 0)
-                        break;
-                    int bIndex = storedTuplePointer.frameIndex;
-                    int tIndex = storedTuplePointer.tupleIndex;
-                    accessorBuild.reset(buffers.get(bIndex));
-                    int c = tpComparator.compare(accessorProbe, i, accessorBuild, tIndex);
-                    if (c == 0) {
-                        boolean predEval = evaluatePredicate(i, tIndex);
-                        if (predEval) {
-                            matchFound = true;
-                            appendToResult(i, tIndex, writer);
-                        }
-                    }
-                } while (true);
-            }
-            if (!matchFound && isLeftOuter) {
-                FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, i,
-                        nullTupleBuild.getFieldEndOffsets(), nullTupleBuild.getByteArray(), 0,
-                        nullTupleBuild.getSize());
-            }
+            join(accessorProbe, i, writer);
         }
     }
 
@@ -155,14 +162,6 @@
                 + Thread.currentThread().getId() + ".");
     }
 
-    private void flushFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
-        writer.nextFrame(buffer);
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
-    }
-
     private boolean evaluatePredicate(int tIx1, int tIx2) {
         if (reverseOutputOrder) { //Role Reversal Optimization is triggered
             return ((predEvaluator == null) || predEvaluator.evaluate(accessorBuild, tIx2, accessorProbe, tIx1));
@@ -172,12 +171,10 @@
     }
 
     private void appendToResult(int probeSidetIx, int buildSidetIx, IFrameWriter writer) throws HyracksDataException {
-        if (!reverseOutputOrder) {
-            FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, probeSidetIx, accessorBuild,
-                    buildSidetIx);
+        if (reverseOutputOrder) {
+            FrameUtils.appendConcatToWriter(writer, appender, accessorBuild, buildSidetIx, accessorProbe, probeSidetIx);
         } else {
-            FrameUtils.appendConcatToWriter(writer, appender, accessorBuild, buildSidetIx, accessorProbe,
-                    probeSidetIx);
+            FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, probeSidetIx, accessorBuild, buildSidetIx);
         }
     }
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index 80b0abe..be8d319f 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -42,7 +42,7 @@
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+import org.apache.hyracks.dataflow.std.util.FrameTuplePairComparator;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
 import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
index 6746b50..2ad89cf 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -20,8 +20,6 @@
 
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
 
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameWriter;
@@ -38,6 +36,11 @@
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.io.RunFileReader;
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.buffermanager.BufferInfo;
+import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
+import org.apache.hyracks.dataflow.std.buffermanager.FrameFreeSlotPolicyFactory;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFrameMemoryManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFramePool;
 
 public class NestedLoopJoin {
     private final FrameTupleAccessor accessorInner;
@@ -46,39 +49,38 @@
     private final ITuplePairComparator tpComparator;
     private final IFrame outBuffer;
     private final IFrame innerBuffer;
-    private final List<ByteBuffer> outBuffers;
-    private final int memSize;
-    private final IHyracksTaskContext ctx;
+    private final VariableFrameMemoryManager outerBufferMngr;
     private RunFileReader runFileReader;
-    private int currentMemSize = 0;
     private final RunFileWriter runFileWriter;
     private final boolean isLeftOuter;
     private final ArrayTupleBuilder nullTupleBuilder;
     private final IPredicateEvaluator predEvaluator;
     private boolean isReversed; //Added for handling correct calling for predicate-evaluator upon recursive calls (in OptimizedHybridHashJoin) that cause role-reversal
+    private BufferInfo tempInfo = new BufferInfo(null, -1, -1);
 
-    public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
-            ITuplePairComparator comparators, int memSize, IPredicateEvaluator predEval, boolean isLeftOuter,
+    public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessorOuter, FrameTupleAccessor accessorInner,
+            ITuplePairComparator comparatorsOuter2Inner, int memSize, IPredicateEvaluator predEval, boolean isLeftOuter,
             INullWriter[] nullWriters1) throws HyracksDataException {
-        this.accessorInner = accessor1;
-        this.accessorOuter = accessor0;
+        this.accessorInner = accessorInner;
+        this.accessorOuter = accessorOuter;
         this.appender = new FrameTupleAppender();
-        this.tpComparator = comparators;
+        this.tpComparator = comparatorsOuter2Inner;
         this.outBuffer = new VSizeFrame(ctx);
         this.innerBuffer = new VSizeFrame(ctx);
         this.appender.reset(outBuffer, true);
-        this.outBuffers = new ArrayList<ByteBuffer>();
-        this.memSize = memSize;
         if (memSize < 3) {
             throw new HyracksDataException("Not enough memory is available for Nested Loop Join");
         }
+        this.outerBufferMngr = new VariableFrameMemoryManager(
+                new VariableFramePool(ctx, ctx.getInitialFrameSize() * (memSize - 2)),
+                FrameFreeSlotPolicyFactory.createFreeSlotPolicy(EnumFreeSlotPolicy.LAST_FIT, memSize - 2));
+
         this.predEvaluator = predEval;
         this.isReversed = false;
-        this.ctx = ctx;
 
         this.isLeftOuter = isLeftOuter;
         if (isLeftOuter) {
-            int innerFieldCount = accessorInner.getFieldCount();
+            int innerFieldCount = this.accessorInner.getFieldCount();
             nullTupleBuilder = new ArrayTupleBuilder(innerFieldCount);
             DataOutput out = nullTupleBuilder.getDataOutput();
             for (int i = 0; i < innerFieldCount; i++) {
@@ -89,8 +91,8 @@
             nullTupleBuilder = null;
         }
 
-        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
-                this.getClass().getSimpleName() + this.toString());
+        FileReference file = ctx.getJobletContext()
+                .createManagedWorkspaceFile(this.getClass().getSimpleName() + this.toString());
         runFileWriter = new RunFileWriter(file, ctx.getIOManager());
         runFileWriter.open();
     }
@@ -100,45 +102,26 @@
     }
 
     public void join(ByteBuffer outerBuffer, IFrameWriter writer) throws HyracksDataException {
-        if (outBuffers.size() < memSize - 3) {
-            createAndCopyFrame(outerBuffer);
-            return;
-        }
-        if (currentMemSize < memSize - 3) {
-            reloadFrame(outerBuffer);
-            return;
-        }
-        runFileReader = runFileWriter.createReader();
-        runFileReader.open();
-        while (runFileReader.nextFrame(innerBuffer)) {
-            for (ByteBuffer outBuffer : outBuffers) {
-                blockJoin(outBuffer, innerBuffer.getBuffer(), writer);
+        if (outerBufferMngr.insertFrame(outerBuffer) < 0) {
+            runFileReader = runFileWriter.createReader();
+            runFileReader.open();
+            while (runFileReader.nextFrame(innerBuffer)) {
+                for (int i = 0; i < outerBufferMngr.getNumFrames(); i++) {
+                    blockJoin(outerBufferMngr.getFrame(i, tempInfo), innerBuffer.getBuffer(), writer);
+                }
+            }
+            runFileReader.close();
+            outerBufferMngr.reset();
+            if (outerBufferMngr.insertFrame(outerBuffer) < 0) {
+                throw new HyracksDataException("The given outer frame of size:" + outerBuffer.capacity()
+                        + " is too big to cache in the buffer. Please choose a larger buffer memory size");
             }
         }
-        runFileReader.close();
-        currentMemSize = 0;
-        reloadFrame(outerBuffer);
     }
 
-    private void createAndCopyFrame(ByteBuffer outerBuffer) throws HyracksDataException {
-        ByteBuffer outerBufferCopy = ctx.allocateFrame(outerBuffer.capacity());
-        FrameUtils.copyAndFlip(outerBuffer, outerBufferCopy);
-        outBuffers.add(outerBufferCopy);
-        currentMemSize++;
-    }
-
-    private void reloadFrame(ByteBuffer outerBuffer) throws HyracksDataException {
-        outBuffers.get(currentMemSize).clear();
-        if (outBuffers.get(currentMemSize).capacity() != outerBuffer.capacity()) {
-            outBuffers.set(currentMemSize, ctx.allocateFrame(outerBuffer.capacity()));
-        }
-        FrameUtils.copyAndFlip(outerBuffer, outBuffers.get(currentMemSize));
-        currentMemSize++;
-    }
-
-    private void blockJoin(ByteBuffer outerBuffer, ByteBuffer innerBuffer, IFrameWriter writer)
+    private void blockJoin(BufferInfo outerBufferInfo, ByteBuffer innerBuffer, IFrameWriter writer)
             throws HyracksDataException {
-        accessorOuter.reset(outerBuffer);
+        accessorOuter.reset(outerBufferInfo.getBuffer(), outerBufferInfo.getStartOffset(), outerBufferInfo.getLength());
         accessorInner.reset(innerBuffer);
         int tupleCount0 = accessorOuter.getTupleCount();
         int tupleCount1 = accessorInner.getTupleCount();
@@ -173,11 +156,10 @@
     }
 
     private void appendToResults(int outerTupleId, int innerTupleId, IFrameWriter writer) throws HyracksDataException {
-        if (!isReversed) {
-            appendResultToFrame(accessorOuter, outerTupleId, accessorInner, innerTupleId, writer);
-        } else {
-            //Role Reversal Optimization is triggered
+        if (isReversed) {
             appendResultToFrame(accessorInner, innerTupleId, accessorOuter, outerTupleId, writer);
+        } else {
+            appendResultToFrame(accessorOuter, outerTupleId, accessorInner, innerTupleId, writer);
         }
     }
 
@@ -196,13 +178,12 @@
         runFileReader = runFileWriter.createDeleteOnCloseReader();
         runFileReader.open();
         while (runFileReader.nextFrame(innerBuffer)) {
-            for (int i = 0; i < currentMemSize; i++) {
-                blockJoin(outBuffers.get(i), innerBuffer.getBuffer(), writer);
+            for (int i = 0; i < outerBufferMngr.getNumFrames(); i++) {
+                blockJoin(outerBufferMngr.getFrame(i, tempInfo), innerBuffer.getBuffer(), writer);
             }
         }
         runFileReader.close();
-        outBuffers.clear();
-        currentMemSize = 0;
+        outerBufferMngr.reset();
 
         appender.write(writer, true);
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 58e1b29..ebcb462 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -19,9 +19,7 @@
 package org.apache.hyracks.dataflow.std.join;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.BitSet;
-import java.util.logging.Logger;
 
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameWriter;
@@ -37,38 +35,42 @@
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
-import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.io.RunFileReader;
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.buffermanager.IPartitionedTupleBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.PreferToSpillFullyOccupiedFramePolicy;
+import org.apache.hyracks.dataflow.std.buffermanager.VPartitionTupleBufferManager;
 import org.apache.hyracks.dataflow.std.structures.ISerializableTable;
 import org.apache.hyracks.dataflow.std.structures.SerializableHashTable;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+import org.apache.hyracks.dataflow.std.util.FrameTuplePairComparator;
 
 /**
- * @author pouria
- *         This class mainly applies one level of HHJ on a pair of
- *         relations. It is always called by the descriptor.
+ * This class mainly applies one level of HHJ on a pair of
+ * relations. It is always called by the descriptor.
  */
 public class OptimizedHybridHashJoin {
 
-    private final int NO_MORE_FREE_BUFFER = -1;
-    private final int END_OF_PARTITION = -1;
-    private final int INVALID_BUFFER = -2;
-    private final int UNALLOCATED_FRAME = -3;
-    private final int BUFFER_FOR_RESIDENT_PARTS = -1;
+    // Used for special probe BigObject which can not be held into the Join memory
+    private FrameTupleAppender bigProbeFrameAppender;
+
+    enum SIDE {
+        BUILD,
+        PROBE
+    }
 
     private IHyracksTaskContext ctx;
 
-    private final String rel0Name;
-    private final String rel1Name;
+    private final String buildRelName;
+    private final String probeRelName;
 
     private final int[] buildKeys;
     private final int[] probeKeys;
 
     private final IBinaryComparator[] comparators;
 
-    private ITuplePartitionComputer buildHpc;
-    private ITuplePartitionComputer probeHpc;
+    private final ITuplePartitionComputer buildHpc;
+    private final ITuplePartitionComputer probeHpc;
 
     private final RecordDescriptor buildRd;
     private final RecordDescriptor probeRd;
@@ -78,83 +80,44 @@
 
     private final IPredicateEvaluator predEvaluator;
     private final boolean isLeftOuter;
-    private final INullWriter[] nullWriters1;
+    private final INullWriter[] nullWriters;
 
-    private IFrame[] memBuffs; //Memory buffers for build
-    private int[] curPBuff; //Current (last) Buffer for each partition
-    private int[] nextBuff; //Next buffer in the partition's buffer chain
-    private int[] buildPSizeInTups; //Size of build partitions (in tuples)
-    private int[] probePSizeInTups; //Size of probe partitions (in tuples)
-    private int nextFreeBuffIx; //Index of next available free buffer to allocate/use
-    private BitSet pStatus; //0=resident, 1=spilled
-    private int numOfPartitions;
-    private int memForJoin;
+    private final BitSet spilledStatus; //0=resident, 1=spilled
+    private final int numOfPartitions;
+    private final int memForJoin;
     private InMemoryHashJoin inMemJoiner; //Used for joining resident partitions
 
+    private IPartitionedTupleBufferManager bufferManager;
+    private PreferToSpillFullyOccupiedFramePolicy spillPolicy;
+
     private final FrameTupleAccessor accessorBuild;
     private final FrameTupleAccessor accessorProbe;
-    private FrameTupleAppender buildTupAppender;
-    private FrameTupleAppender probeTupAppenderToResident;
-    private FrameTupleAppender probeTupAppenderToSpilled;
 
-    private int numOfSpilledParts;
-    private IFrame[] sPartBuffs; //Buffers for probe spilled partitions (one buffer per spilled partition)
-    private IFrame probeResBuff; //Buffer for probe resident partition tuples
-    private IFrame reloadBuffer; //Buffer for reloading spilled partitions during partition tuning
-
-    private int[] buildPSizeInFrames; //Used for partition tuning
-    private int freeFramesCounter; //Used for partition tuning
-
-    private boolean isTableEmpty; //Added for handling the case, where build side is empty (tableSize is 0)
     private boolean isReversed; //Added for handling correct calling for predicate-evaluator upon recursive calls that cause role-reversal
 
-    private static final Logger LOGGER = Logger.getLogger(OptimizedHybridHashJoin.class.getName());
+    // stats information
+    private int[] buildPSizeInTups;
+    private IFrame reloadBuffer;
+    private TuplePointer tempPtr = new TuplePointer(); // this is a reusable object to store the pointer,which is not used anywhere.
+                                                       // we mainly use it to match the corresponding function signature.
+    private int[] probePSizeInTups;
 
-    public OptimizedHybridHashJoin(IHyracksTaskContext ctx, int memForJoin, int numOfPartitions, String rel0Name,
-            String rel1Name, int[] keys0, int[] keys1, IBinaryComparator[] comparators, RecordDescriptor buildRd,
-            RecordDescriptor probeRd, ITuplePartitionComputer probeHpc, ITuplePartitionComputer buildHpc,
-            IPredicateEvaluator predEval) {
+    public OptimizedHybridHashJoin(IHyracksTaskContext ctx, int memForJoin, int numOfPartitions, String probeRelName,
+            String buildRelName, int[] probeKeys, int[] buildKeys, IBinaryComparator[] comparators,
+            RecordDescriptor probeRd, RecordDescriptor buildRd, ITuplePartitionComputer probeHpc,
+            ITuplePartitionComputer buildHpc, IPredicateEvaluator predEval, boolean isLeftOuter,
+            INullWriterFactory[] nullWriterFactories1) {
         this.ctx = ctx;
         this.memForJoin = memForJoin;
         this.buildRd = buildRd;
         this.probeRd = probeRd;
         this.buildHpc = buildHpc;
         this.probeHpc = probeHpc;
-        this.buildKeys = keys1;
-        this.probeKeys = keys0;
+        this.buildKeys = buildKeys;
+        this.probeKeys = probeKeys;
         this.comparators = comparators;
-        this.rel0Name = rel0Name;
-        this.rel1Name = rel1Name;
-
-        this.numOfPartitions = numOfPartitions;
-        this.buildRFWriters = new RunFileWriter[numOfPartitions];
-        this.probeRFWriters = new RunFileWriter[numOfPartitions];
-
-        this.accessorBuild = new FrameTupleAccessor(buildRd);
-        this.accessorProbe = new FrameTupleAccessor(probeRd);
-
-        this.predEvaluator = predEval;
-        this.isLeftOuter = false;
-        this.nullWriters1 = null;
-        this.isReversed = false;
-
-    }
-
-    public OptimizedHybridHashJoin(IHyracksTaskContext ctx, int memForJoin, int numOfPartitions, String rel0Name,
-            String rel1Name, int[] keys0, int[] keys1, IBinaryComparator[] comparators, RecordDescriptor buildRd,
-            RecordDescriptor probeRd, ITuplePartitionComputer probeHpc, ITuplePartitionComputer buildHpc,
-            IPredicateEvaluator predEval, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
-        this.ctx = ctx;
-        this.memForJoin = memForJoin;
-        this.buildRd = buildRd;
-        this.probeRd = probeRd;
-        this.buildHpc = buildHpc;
-        this.probeHpc = probeHpc;
-        this.buildKeys = keys1;
-        this.probeKeys = keys0;
-        this.comparators = comparators;
-        this.rel0Name = rel0Name;
-        this.rel1Name = rel1Name;
+        this.buildRelName = buildRelName;
+        this.probeRelName = probeRelName;
 
         this.numOfPartitions = numOfPartitions;
         this.buildRFWriters = new RunFileWriter[numOfPartitions];
@@ -167,49 +130,30 @@
         this.isLeftOuter = isLeftOuter;
         this.isReversed = false;
 
-        this.nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
+        this.spilledStatus = new BitSet(numOfPartitions);
+
+        this.nullWriters = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
         if (isLeftOuter) {
             for (int i = 0; i < nullWriterFactories1.length; i++) {
-                nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
+                nullWriters[i] = nullWriterFactories1[i].createNullWriter();
             }
         }
     }
 
     public void initBuild() throws HyracksDataException {
-        memBuffs = new IFrame[memForJoin];
-        curPBuff = new int[numOfPartitions];
-        nextBuff = new int[memForJoin];
-        pStatus = new BitSet(numOfPartitions);
+        bufferManager = new VPartitionTupleBufferManager(ctx,
+                PreferToSpillFullyOccupiedFramePolicy.createAtMostOneFrameForSpilledPartitionConstrain(spilledStatus),
+                numOfPartitions, memForJoin * ctx.getInitialFrameSize());
+        spillPolicy = new PreferToSpillFullyOccupiedFramePolicy(bufferManager, spilledStatus,
+                ctx.getInitialFrameSize());
+        spilledStatus.clear();
         buildPSizeInTups = new int[numOfPartitions];
-
-        buildPSizeInFrames = new int[numOfPartitions];
-        freeFramesCounter = memForJoin - numOfPartitions;
-
-        for (int i = 0; i < numOfPartitions; i++) { //Allocating one buffer per partition and setting as the head of the chain of buffers for that partition
-            memBuffs[i] = new VSizeFrame(ctx);
-            curPBuff[i] = i;
-            nextBuff[i] = -1;
-            buildPSizeInFrames[i] = 1; //The dedicated initial buffer
-        }
-
-        nextFreeBuffIx = ((numOfPartitions < memForJoin) ? numOfPartitions : NO_MORE_FREE_BUFFER); //Setting the chain of unallocated frames
-        for (int i = numOfPartitions; i < memBuffs.length; i++) {
-            nextBuff[i] = UNALLOCATED_FRAME;
-        }
-
-        buildTupAppender = new FrameTupleAppender();
-
     }
 
     public void build(ByteBuffer buffer) throws HyracksDataException {
         accessorBuild.reset(buffer);
         int tupleCount = accessorBuild.getTupleCount();
 
-        boolean print = false;
-        if (print) {
-            accessorBuild.prettyPrint();
-        }
-
         for (int i = 0; i < tupleCount; ++i) {
             int pid = buildHpc.partition(accessorBuild, i, numOfPartitions);
             processTuple(i, pid);
@@ -219,304 +163,198 @@
     }
 
     private void processTuple(int tid, int pid) throws HyracksDataException {
-        IFrame partition = memBuffs[curPBuff[pid]]; //Getting current buffer for the target partition
-
-        if (!pStatus.get(pid)) { //resident partition
-            buildTupAppender.reset(partition, false);
-            while (true) {
-                if (buildTupAppender.append(accessorBuild, tid)) { //Tuple added to resident partition successfully
-                    break;
-                }
-                //partition does not have enough room
-                int newBuffIx = allocateFreeBuffer(pid);
-                if (newBuffIx == NO_MORE_FREE_BUFFER) { //Spill one partition
-                    int pidToSpill = selectPartitionToSpill();
-                    if (pidToSpill == -1) { //No more partition to spill
-                        throw new HyracksDataException("not enough memory for Hash Join (Allocation exceeds the limit)");
-                    }
-                    spillPartition(pidToSpill);
-                    buildTupAppender.reset(memBuffs[pidToSpill], true);
-                    processTuple(tid, pid);
-                    break;
-                } //New Buffer allocated successfully
-                partition = memBuffs[curPBuff[pid]]; //Current Buffer for the partition is now updated by allocateFreeBuffer() call above
-                buildTupAppender.reset(partition, true);
-                if (!buildTupAppender.append(accessorBuild, tid)) {
-                    throw new HyracksDataException("Invalid State (Can not append to newly allocated buffer)");
-                }
-                buildPSizeInFrames[pid]++;
-                break;
-            }
-        } else { //spilled partition
-            boolean needClear = false;
-            while (true) {
-                buildTupAppender.reset(partition, needClear);
-                if (buildTupAppender.append(accessorBuild, tid)) {
-                    break;
-                }
-                //Dedicated in-memory buffer for the partition is full, needed to be flushed first
-                buildWrite(pid, partition.getBuffer());
-                partition.reset();
-                needClear = true;
-                buildPSizeInFrames[pid]++;
-            }
+        while (!bufferManager.insertTuple(pid, accessorBuild, tid, tempPtr)) {
+            selectAndSpillVictim(pid);
         }
     }
 
-    private int allocateFreeBuffer(int pid) throws HyracksDataException {
-        if (nextFreeBuffIx != NO_MORE_FREE_BUFFER) {
-            if (memBuffs[nextFreeBuffIx] == null) {
-                memBuffs[nextFreeBuffIx] = new VSizeFrame(ctx);
-            }
-            int curPartBuffIx = curPBuff[pid];
-            curPBuff[pid] = nextFreeBuffIx;
-            int oldNext = nextBuff[nextFreeBuffIx];
-            nextBuff[nextFreeBuffIx] = curPartBuffIx;
-            if (oldNext == UNALLOCATED_FRAME) {
-                nextFreeBuffIx++;
-                if (nextFreeBuffIx == memForJoin) { //No more free buffer
-                    nextFreeBuffIx = NO_MORE_FREE_BUFFER;
-                }
-            } else {
-                nextFreeBuffIx = oldNext;
-            }
-            memBuffs[curPBuff[pid]].reset();
-
-            freeFramesCounter--;
-            return (curPBuff[pid]);
-        } else {
-            return NO_MORE_FREE_BUFFER; //A partitions needs to be spilled (if feasible)
+    private void selectAndSpillVictim(int pid) throws HyracksDataException {
+        int victimPartition = spillPolicy.selectVictimPartition(pid);
+        if (victimPartition < 0) {
+            throw new HyracksDataException(
+                    "No more space left in the memory buffer, please give join more memory budgets.");
         }
-    }
-
-    private int selectPartitionToSpill() {
-        int maxSize = -1;
-        int partitionToSpill = -1;
-        for (int i = 0; i < buildPSizeInTups.length; i++) { //Find the largest partition, to spill
-            if (!pStatus.get(i) && (buildPSizeInTups[i] > maxSize)) {
-                maxSize = buildPSizeInTups[i];
-                partitionToSpill = i;
-            }
-        }
-        return partitionToSpill;
+        spillPartition(victimPartition);
     }
 
     private void spillPartition(int pid) throws HyracksDataException {
-        LOGGER.fine("OptimizedHybridHashJoin is spilling partition:" + pid + " with " + buildPSizeInFrames[pid]
-                + " frames for Thread ID " + Thread.currentThread().getId() + " (free frames: " + freeFramesCounter
-                + ").");
-        int curBuffIx = curPBuff[pid];
-        while (curBuffIx != END_OF_PARTITION) {
-            IFrame frame = memBuffs[curBuffIx];
-            buildWrite(pid, frame.getBuffer());
-            frame.reset();
-
-            int freedBuffIx = curBuffIx;
-            curBuffIx = nextBuff[curBuffIx];
-
-            if (freedBuffIx != pid) {
-                nextBuff[freedBuffIx] = nextFreeBuffIx;
-                nextFreeBuffIx = freedBuffIx;
-                freeFramesCounter++;
-            }
-        }
-        curPBuff[pid] = pid;
-        pStatus.set(pid);
-        LOGGER.fine("OptimizedHybridHashJoin has freed " + freeFramesCounter + " frames by spilling partition:" + pid
-                + " for Thread ID " + Thread.currentThread().getId() + ".");
+        RunFileWriter writer = getSpillWriterOrCreateNewOneIfNotExist(pid, SIDE.BUILD);
+        bufferManager.flushPartition(pid, writer);
+        bufferManager.clearPartition(pid);
+        spilledStatus.set(pid);
     }
 
-    private void buildWrite(int pid, ByteBuffer buff) throws HyracksDataException {
-        RunFileWriter writer = buildRFWriters[pid];
+    private RunFileWriter getSpillWriterOrCreateNewOneIfNotExist(int pid, SIDE whichSide) throws HyracksDataException {
+        RunFileWriter[] runFileWriters = null;
+        String refName = null;
+        switch (whichSide) {
+            case BUILD:
+                runFileWriters = buildRFWriters;
+                refName = buildRelName;
+                break;
+            case PROBE:
+                refName = probeRelName;
+                runFileWriters = probeRFWriters;
+                break;
+        }
+        RunFileWriter writer = runFileWriters[pid];
         if (writer == null) {
-            FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(rel0Name);
-            LOGGER.fine("OptimizedHybridHashJoin is creating a run file (" + file.getFile().getAbsolutePath()
-                    + ") for partition:" + pid + " for Thread ID " + Thread.currentThread().getId() + ".");
+            FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(refName);
             writer = new RunFileWriter(file, ctx.getIOManager());
             writer.open();
-            buildRFWriters[pid] = writer;
+            runFileWriters[pid] = writer;
         }
-        writer.nextFrame(buff);
+        return writer;
     }
 
     public void closeBuild() throws HyracksDataException {
-        for (int i = 0; i < numOfPartitions; i++) { //Remove Empty Partitions' allocated frame
-            if (buildPSizeInTups[i] == 0) {
-                buildPSizeInFrames[i]--;
-                nextBuff[curPBuff[i]] = nextFreeBuffIx;
-                nextFreeBuffIx = curPBuff[i];
-                curPBuff[i] = INVALID_BUFFER;
-                freeFramesCounter++;
-            }
-        }
 
-        ByteBuffer buff = null;
-        for (int i = pStatus.nextSetBit(0); i >= 0; i = pStatus.nextSetBit(i + 1)) { //flushing and DeAllocating the dedicated buffers for the spilled partitions
-            buff = memBuffs[i].getBuffer();
-            accessorBuild.reset(buff);
-            if (accessorBuild.getTupleCount() > 0) {
-                buildWrite(i, buff);
-                buildPSizeInFrames[i]++;
-            }
-            nextBuff[i] = nextFreeBuffIx;
-            nextFreeBuffIx = i;
-            freeFramesCounter++;
-            curPBuff[i] = INVALID_BUFFER;
+        closeAllSpilledPartitions(SIDE.BUILD);
 
-            if (buildRFWriters[i] != null) {
-                buildRFWriters[i].close();
-            }
-        }
-
-        partitionTune(); //Trying to bring back as many spilled partitions as possible, making them resident
+        bringBackSpilledPartitionIfHasMoreMemory(); //Trying to bring back as many spilled partitions as possible, making them resident
 
         int inMemTupCount = 0;
-        int inMemFrameCount = 0;
-        int spilledFrameCount = 0;
-        numOfSpilledParts = 0;
 
-        for (int i = 0; i < numOfPartitions; i++) {
-            if (!pStatus.get(i)) {
-                inMemTupCount += buildPSizeInTups[i];
-                inMemFrameCount += buildPSizeInFrames[i];
-            } else {
-                spilledFrameCount += buildPSizeInFrames[i];
-                numOfSpilledParts++;
-            }
+        for (int i = spilledStatus.nextClearBit(0); i >= 0
+                && i < numOfPartitions; i = spilledStatus.nextClearBit(i + 1)) {
+            inMemTupCount += buildPSizeInTups[i];
         }
 
-        LOGGER.fine("OptimizedHybridHashJoin build phase has spilled " + numOfSpilledParts + " of " + numOfPartitions
-                + " partitions for Thread ID " + Thread.currentThread().getId() + ". (" + inMemFrameCount
-                + " in-memory frames, " + spilledFrameCount + " spilled frames)");
         createInMemoryJoiner(inMemTupCount);
         cacheInMemJoin();
-        this.isTableEmpty = (inMemTupCount == 0);
     }
 
-    private void partitionTune() throws HyracksDataException {
-        reloadBuffer = new VSizeFrame(ctx);
-        ArrayList<Integer> reloadSet = selectPartitionsToReload();
-        for (int i = 0; i < reloadSet.size(); i++) {
-            int pid = reloadSet.get(i);
-            int[] buffsToLoad = new int[buildPSizeInFrames[pid]];
-            for (int j = 0; j < buffsToLoad.length; j++) {
-                buffsToLoad[j] = nextFreeBuffIx;
-                int oldNext = nextBuff[nextFreeBuffIx];
-                if (oldNext == UNALLOCATED_FRAME) {
-                    nextFreeBuffIx++;
-                    if (nextFreeBuffIx == memForJoin) { //No more free buffer
-                        nextFreeBuffIx = NO_MORE_FREE_BUFFER;
-                    }
-                } else {
-                    nextFreeBuffIx = oldNext;
-                }
-
+    /**
+     * In case of failure happens, we need to clear up the generated temporary files.
+     */
+    public void clearBuildTempFiles() {
+        for (int i = 0; i < buildRFWriters.length; i++) {
+            if (buildRFWriters[i] != null) {
+                buildRFWriters[i].getFileReference().delete();
             }
-            curPBuff[pid] = buffsToLoad[0];
-            for (int k = 1; k < buffsToLoad.length; k++) {
-                nextBuff[buffsToLoad[k - 1]] = buffsToLoad[k];
-            }
-            loadPartitionInMem(pid, buildRFWriters[pid], buffsToLoad);
         }
-        reloadSet.clear();
     }
 
-    private void loadPartitionInMem(int pid, RunFileWriter wr, int[] buffs) throws HyracksDataException {
+    private void closeAllSpilledPartitions(SIDE whichSide) throws HyracksDataException {
+        RunFileWriter[] runFileWriters = null;
+        switch (whichSide) {
+            case BUILD:
+                runFileWriters = buildRFWriters;
+                break;
+            case PROBE:
+                runFileWriters = probeRFWriters;
+                break;
+        }
+
+        for (int pid = spilledStatus.nextSetBit(0); pid >= 0; pid = spilledStatus.nextSetBit(pid + 1)) {
+            if (bufferManager.getNumTuples(pid) > 0) {
+                bufferManager.flushPartition(pid, getSpillWriterOrCreateNewOneIfNotExist(pid, whichSide));
+                bufferManager.clearPartition(pid);
+                runFileWriters[pid].close();
+            }
+        }
+    }
+
+    private void bringBackSpilledPartitionIfHasMoreMemory() throws HyracksDataException {
+        // we need number of |spilledPartitions| buffers to store the probe data
+        int freeSpace = (memForJoin - spilledStatus.cardinality()) * ctx.getInitialFrameSize();
+        for (int p = spilledStatus.nextClearBit(0); p >= 0
+                && p < numOfPartitions; p = spilledStatus.nextClearBit(p + 1)) {
+            freeSpace -= bufferManager.getPhysicalSize(p);
+        }
+
+        int pid = 0;
+        while ((pid = selectPartitionsToReload(freeSpace, pid)) >= 0) {
+            if (!loadPartitionInMem(pid, buildRFWriters[pid])) {
+                return;
+            }
+            freeSpace -= bufferManager.getPhysicalSize(pid);
+        }
+    }
+
+    private boolean loadPartitionInMem(int pid, RunFileWriter wr) throws HyracksDataException {
         RunFileReader r = wr.createDeleteOnCloseReader();
         r.open();
-        int counter = 0;
-        ByteBuffer mBuff = null;
-        reloadBuffer.reset();
-        while (r.nextFrame(reloadBuffer)) {
-            if (memBuffs[buffs[counter]] == null) {
-                memBuffs[buffs[counter]] = new VSizeFrame(ctx);
-            }
-            memBuffs[buffs[counter]].ensureFrameSize(reloadBuffer.getFrameSize());
-            mBuff = memBuffs[buffs[counter]].getBuffer();
-            FrameUtils.copyAndFlip(reloadBuffer.getBuffer(), mBuff);
-            counter++;
-            reloadBuffer.reset();
+        if (reloadBuffer == null) {
+            reloadBuffer = new VSizeFrame(ctx);
         }
-
-        int curNext = nextBuff[buffs[buffs.length - 1]];
-        nextBuff[buffs[buffs.length - 1]] = END_OF_PARTITION;
-        nextFreeBuffIx = curNext;
+        while (r.nextFrame(reloadBuffer)) {
+            accessorBuild.reset(reloadBuffer.getBuffer());
+            for (int tid = 0; tid < accessorBuild.getTupleCount(); tid++) {
+                if (!bufferManager.insertTuple(pid, accessorBuild, tid, tempPtr)) {
+                    // for some reason (e.g. due to fragmentation) if the inserting failed, we need to clear the occupied frames
+                    bufferManager.clearPartition(pid);
+                    r.close();
+                    return false;
+                }
+            }
+        }
 
         r.close();
-        pStatus.set(pid, false);
+        spilledStatus.set(pid, false);
         buildRFWriters[pid] = null;
+        return true;
     }
 
-    private ArrayList<Integer> selectPartitionsToReload() {
-        ArrayList<Integer> p = new ArrayList<Integer>();
-        for (int i = pStatus.nextSetBit(0); i >= 0; i = pStatus.nextSetBit(i + 1)) {
-            if (buildPSizeInFrames[i] > 0 && (freeFramesCounter - buildPSizeInFrames[i] >= 0)) {
-                p.add(i);
-                freeFramesCounter -= buildPSizeInFrames[i];
-            }
-            if (freeFramesCounter < 1) { //No more free buffer available
-                return p;
+    private int selectPartitionsToReload(int freeSpace, int pid) {
+        for (int i = spilledStatus.nextSetBit(pid); i >= 0; i = spilledStatus.nextSetBit(i + 1)) {
+            assert buildRFWriters[i].getFileSize() > 0 : "How comes a spilled partition have size 0?";
+            if (freeSpace >= buildRFWriters[i].getFileSize()) {
+                return i;
             }
         }
-        return p;
+        return -1;
     }
 
     private void createInMemoryJoiner(int inMemTupCount) throws HyracksDataException {
         ISerializableTable table = new SerializableHashTable(inMemTupCount, ctx);
         this.inMemJoiner = new InMemoryHashJoin(ctx, inMemTupCount, new FrameTupleAccessor(probeRd), probeHpc,
-                new FrameTupleAccessor(buildRd), buildHpc, new FrameTuplePairComparator(probeKeys, buildKeys,
-                        comparators), isLeftOuter, nullWriters1, table, predEvaluator, isReversed);
+                new FrameTupleAccessor(buildRd), buildHpc,
+                new FrameTuplePairComparator(probeKeys, buildKeys, comparators), isLeftOuter, nullWriters, table,
+                predEvaluator, isReversed);
     }
 
     private void cacheInMemJoin() throws HyracksDataException {
 
         for (int pid = 0; pid < numOfPartitions; pid++) {
-            if (!pStatus.get(pid)) {
-                int nextBuffIx = curPBuff[pid];
-                while (nextBuffIx > -1) { //It is not Invalid or End_Of_Partition
-                    inMemJoiner.build(memBuffs[nextBuffIx].getBuffer());
-                    nextBuffIx = nextBuff[nextBuffIx];
-                }
+            if (!spilledStatus.get(pid)) {
+                bufferManager.flushPartition(pid, new IFrameWriter() {
+                    @Override
+                    public void open() throws HyracksDataException {
+
+                    }
+
+                    @Override
+                    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                        inMemJoiner.build(buffer);
+                    }
+
+                    @Override
+                    public void fail() throws HyracksDataException {
+
+                    }
+
+                    @Override
+                    public void close() throws HyracksDataException {
+
+                    }
+                });
             }
         }
     }
 
     public void initProbe() throws HyracksDataException {
 
-        sPartBuffs = new IFrame[numOfSpilledParts];
-        for (int i = 0; i < numOfSpilledParts; i++) {
-            sPartBuffs[i] = new VSizeFrame(ctx);
-        }
-        curPBuff = new int[numOfPartitions];
-        int nextBuffIxToAlloc = 0;
-        /* We only need to allocate one frame per spilled partition.
-         * Resident partitions do not need frames in probe, as their tuples join
-         * immediately with the resident build tuples using the inMemoryHashJoin */
-        for (int i = 0; i < numOfPartitions; i++) {
-            curPBuff[i] = (pStatus.get(i)) ? nextBuffIxToAlloc++ : BUFFER_FOR_RESIDENT_PARTS;
-        }
         probePSizeInTups = new int[numOfPartitions];
         probeRFWriters = new RunFileWriter[numOfPartitions];
 
-        probeResBuff = new VSizeFrame(ctx);
-
-        probeTupAppenderToResident = new FrameTupleAppender();
-        probeTupAppenderToResident.reset(probeResBuff, true);
-
-        probeTupAppenderToSpilled = new FrameTupleAppender();
-
     }
 
     public void probe(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
         accessorProbe.reset(buffer);
         int tupleCount = accessorProbe.getTupleCount();
 
-        boolean print = false;
-        if (print) {
-            accessorProbe.prettyPrint();
-        }
-
-        if (numOfSpilledParts == 0) {
+        if (isBuildRelAllInMemory()) {
             inMemJoiner.join(buffer, writer);
             return;
         }
@@ -524,64 +362,60 @@
             int pid = probeHpc.partition(accessorProbe, i, numOfPartitions);
 
             if (buildPSizeInTups[pid] > 0 || isLeftOuter) { //Tuple has potential match from previous phase
-                if (pStatus.get(pid)) { //pid is Spilled
-                    boolean needToClear = false;
-                    IFrame frame = sPartBuffs[curPBuff[pid]];
-                    while (true) {
-                        probeTupAppenderToSpilled.reset(frame, needToClear);
-                        if (probeTupAppenderToSpilled.append(accessorProbe, i)) {
+                if (spilledStatus.get(pid)) { //pid is Spilled
+                    while (!bufferManager.insertTuple(pid, accessorProbe, i, tempPtr)) {
+                        int victim = pid;
+                        if (bufferManager.getNumTuples(pid) == 0) { // current pid is empty, choose the biggest one
+                            victim = spillPolicy.findSpilledPartitionWithMaxMemoryUsage();
+                        }
+                        if (victim < 0) { // current tuple is too big for all the free space
+                            flushBigProbeObjectToDisk(pid, accessorProbe, i);
                             break;
                         }
-                        probeWrite(pid, frame.getBuffer());
-                        frame.reset();
-                        needToClear = true;
+                        RunFileWriter runFileWriter = getSpillWriterOrCreateNewOneIfNotExist(victim, SIDE.PROBE);
+                        bufferManager.flushPartition(victim, runFileWriter);
+                        bufferManager.clearPartition(victim);
                     }
                 } else { //pid is Resident
-                    while (true) {
-                        if (probeTupAppenderToResident.append(accessorProbe, i)) {
-                            break;
-                        }
-                        inMemJoiner.join(probeResBuff.getBuffer(), writer);
-                        probeTupAppenderToResident.reset(probeResBuff, true);
-                    }
-
+                    inMemJoiner.join(accessorProbe, i, writer);
                 }
                 probePSizeInTups[pid]++;
             }
-
         }
 
     }
 
-    public void closeProbe(IFrameWriter writer) throws HyracksDataException { //We do NOT join the spilled partitions here, that decision is made at the descriptor level (which join technique to use)
-        inMemJoiner.join(probeResBuff.getBuffer(), writer);
+    private void flushBigProbeObjectToDisk(int pid, FrameTupleAccessor accessorProbe, int i)
+            throws HyracksDataException {
+        if (bigProbeFrameAppender == null) {
+            bigProbeFrameAppender = new FrameTupleAppender(new VSizeFrame(ctx));
+        }
+        RunFileWriter runFileWriter = getSpillWriterOrCreateNewOneIfNotExist(pid, SIDE.PROBE);
+        if (!bigProbeFrameAppender.append(accessorProbe, i)) {
+            throw new HyracksDataException("The given tuple is too big");
+        }
+        bigProbeFrameAppender.write(runFileWriter, true);
+    }
+
+    private boolean isBuildRelAllInMemory() {
+        return spilledStatus.nextSetBit(0) < 0;
+    }
+
+    public void closeProbe(IFrameWriter writer) throws HyracksDataException {
+        //We do NOT join the spilled partitions here, that decision is made at the descriptor level (which join technique to use)
         inMemJoiner.closeJoin(writer);
+        closeAllSpilledPartitions(SIDE.PROBE);
+        bufferManager = null;
+    }
 
-        for (int pid = pStatus.nextSetBit(0); pid >= 0; pid = pStatus.nextSetBit(pid + 1)) {
-            ByteBuffer buff = sPartBuffs[curPBuff[pid]].getBuffer();
-            accessorProbe.reset(buff);
-            if (accessorProbe.getTupleCount() > 0) {
-                probeWrite(pid, buff);
+    /**
+     * In case of failure happens, we need to clear up the generated temporary files.
+     */
+    public void clearProbeTempFiles() {
+        for (int i = 0; i < probeRFWriters.length; i++) {
+            if (probeRFWriters[i] != null) {
+                probeRFWriters[i].getFileReference().delete();
             }
-            closeProbeWriter(pid);
-        }
-    }
-
-    private void probeWrite(int pid, ByteBuffer buff) throws HyracksDataException {
-        RunFileWriter pWriter = probeRFWriters[pid];
-        if (pWriter == null) {
-            FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(rel1Name);
-            pWriter = new RunFileWriter(file, ctx.getIOManager());
-            pWriter.open();
-            probeRFWriters[pid] = pWriter;
-        }
-        pWriter.nextFrame(buff);
-    }
-
-    private void closeProbeWriter(int pid) throws HyracksDataException {
-        RunFileWriter writer = probeRFWriters[pid];
-        if (writer != null) {
-            writer.close();
         }
     }
 
@@ -589,10 +423,6 @@
         return ((buildRFWriters[pid] == null) ? null : (buildRFWriters[pid]).createDeleteOnCloseReader());
     }
 
-    public long getBuildPartitionSize(int pid) {
-        return ((buildRFWriters[pid] == null) ? 0 : buildRFWriters[pid].getFileSize());
-    }
-
     public int getBuildPartitionSizeInTup(int pid) {
         return (buildPSizeInTups[pid]);
     }
@@ -601,10 +431,6 @@
         return ((probeRFWriters[pid] == null) ? null : (probeRFWriters[pid]).createDeleteOnCloseReader());
     }
 
-    public long getProbePartitionSize(int pid) {
-        return ((probeRFWriters[pid] == null) ? 0 : probeRFWriters[pid].getFileSize());
-    }
-
     public int getProbePartitionSizeInTup(int pid) {
         return (probePSizeInTups[pid]);
     }
@@ -630,37 +456,7 @@
     }
 
     public BitSet getPartitionStatus() {
-        return pStatus;
-    }
-
-    public String debugGetStats() {
-        int numOfResidentPartitions = 0;
-        int numOfSpilledPartitions = 0;
-        double sumOfBuildSpilledSizes = 0;
-        double sumOfProbeSpilledSizes = 0;
-        int numOfInMemTups = 0;
-        for (int i = 0; i < numOfPartitions; i++) {
-            if (pStatus.get(i)) { //Spilled
-                numOfSpilledPartitions++;
-                sumOfBuildSpilledSizes += buildPSizeInTups[i];
-                sumOfProbeSpilledSizes += probePSizeInTups[i];
-            } else { //Resident
-                numOfResidentPartitions++;
-                numOfInMemTups += buildPSizeInTups[i];
-            }
-        }
-
-        double avgBuildSpSz = sumOfBuildSpilledSizes / numOfSpilledPartitions;
-        double avgProbeSpSz = sumOfProbeSpilledSizes / numOfSpilledPartitions;
-        String s = "Resident Partitions:\t" + numOfResidentPartitions + "\nSpilled Partitions:\t"
-                + numOfSpilledPartitions + "\nAvg Build Spilled Size:\t" + avgBuildSpSz + "\nAvg Probe Spilled Size:\t"
-                + avgProbeSpSz + "\nIn-Memory Tups:\t" + numOfInMemTups + "\nNum of Free Buffers:\t"
-                + freeFramesCounter;
-        return s;
-    }
-
-    public boolean isTableEmpty() {
-        return this.isTableEmpty;
+        return spilledStatus;
     }
 
     public void setIsReversed(boolean b) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 07fdde1..22ad91f 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -23,6 +23,7 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.BitSet;
+import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.hyracks.api.comm.IFrame;
@@ -47,8 +48,9 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFamily;
 import org.apache.hyracks.dataflow.common.io.RunFileReader;
@@ -59,6 +61,7 @@
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
 import org.apache.hyracks.dataflow.std.structures.ISerializableTable;
 import org.apache.hyracks.dataflow.std.structures.SerializableHashTable;
+import org.apache.hyracks.dataflow.std.util.FrameTuplePairComparator;
 
 /**
  * @author pouria
@@ -109,15 +112,15 @@
     private static final String PROBE_REL = "RelR";
     private static final String BUILD_REL = "RelS";
 
-    private final int memsize;
+    private final int frameLimit;
     private final int inputsize0;
     private final double fudgeFactor;
     private final int[] probeKeys;
     private final int[] buildKeys;
     private final IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories;
     private final IBinaryComparatorFactory[] comparatorFactories; //For in-mem HJ
-    private final ITuplePairComparatorFactory tuplePairComparatorFactory0; //For NLJ in probe
-    private final ITuplePairComparatorFactory tuplePairComparatorFactory1; //For NLJ in probe
+    private final ITuplePairComparatorFactory tuplePairComparatorFactoryProbe2Build; //For NLJ in probe
+    private final ITuplePairComparatorFactory tuplePairComparatorFactoryBuild2Probe; //For NLJ in probe
     private final IPredicateEvaluatorFactory predEvaluatorFactory;
 
     private final boolean isLeftOuter;
@@ -130,50 +133,38 @@
 
     private static final Logger LOGGER = Logger.getLogger(OptimizedHybridHashJoinOperatorDescriptor.class.getName());
 
-    public OptimizedHybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+    public OptimizedHybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int frameLimit, int inputsize0,
             double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories,
             IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor,
-            ITuplePairComparatorFactory tupPaircomparatorFactory0,
-            ITuplePairComparatorFactory tupPaircomparatorFactory1, IPredicateEvaluatorFactory predEvaluatorFactory,
+            ITuplePairComparatorFactory tupPaircomparatorFactory01,
+            ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvaluatorFactory,
             boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) throws HyracksDataException {
 
         super(spec, 2, 1);
+        this.frameLimit = frameLimit;
         this.inputsize0 = inputsize0;
-        this.memsize = memsize;
         this.fudgeFactor = factor;
         this.probeKeys = keys0;
         this.buildKeys = keys1;
         this.hashFunctionGeneratorFactories = hashFunctionGeneratorFactories;
         this.comparatorFactories = comparatorFactories;
-        this.tuplePairComparatorFactory0 = tupPaircomparatorFactory0;
-        this.tuplePairComparatorFactory1 = tupPaircomparatorFactory1;
+        this.tuplePairComparatorFactoryProbe2Build = tupPaircomparatorFactory01;
+        this.tuplePairComparatorFactoryBuild2Probe = tupPaircomparatorFactory10;
         recordDescriptors[0] = recordDescriptor;
         this.predEvaluatorFactory = predEvaluatorFactory;
         this.isLeftOuter = isLeftOuter;
         this.nullWriterFactories1 = nullWriterFactories1;
     }
 
-    public OptimizedHybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+    public OptimizedHybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int frameLimit, int inputsize0,
             double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories,
             IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor,
-            ITuplePairComparatorFactory tupPaircomparatorFactory0,
-            ITuplePairComparatorFactory tupPaircomparatorFactory1, IPredicateEvaluatorFactory predEvaluatorFactory)
-                    throws HyracksDataException {
-
-        super(spec, 2, 1);
-        this.memsize = memsize;
-        this.inputsize0 = inputsize0;
-        this.fudgeFactor = factor;
-        this.probeKeys = keys0;
-        this.buildKeys = keys1;
-        this.hashFunctionGeneratorFactories = hashFunctionGeneratorFactories;
-        this.comparatorFactories = comparatorFactories;
-        this.tuplePairComparatorFactory0 = tupPaircomparatorFactory0;
-        this.tuplePairComparatorFactory1 = tupPaircomparatorFactory1;
-        this.predEvaluatorFactory = predEvaluatorFactory;
-        recordDescriptors[0] = recordDescriptor;
-        this.isLeftOuter = false;
-        this.nullWriterFactories1 = null;
+            ITuplePairComparatorFactory tupPaircomparatorFactory01,
+            ITuplePairComparatorFactory tupPaircomparatorFactory10, IPredicateEvaluatorFactory predEvaluatorFactory)
+            throws HyracksDataException {
+        this(spec, frameLimit, inputsize0, factor, keys0, keys1, hashFunctionGeneratorFactories,
+                comparatorFactories, recordDescriptor, tupPaircomparatorFactory01, tupPaircomparatorFactory10,
+                predEvaluatorFactory, false, null);
     }
 
     @Override
@@ -262,8 +253,8 @@
         public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
                 IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
 
-            final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
-            final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(probeAid, 0);
+            final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(probeAid, 0);
 
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
             for (int i = 0; i < comparatorFactories.length; i++) {
@@ -281,28 +272,25 @@
                         hashFunctionGeneratorFactories).createPartitioner(0);
                 ITuplePartitionComputer buildHpc = new FieldHashPartitionComputerFamily(buildKeys,
                         hashFunctionGeneratorFactories).createPartitioner(0);
+                boolean isFailed = false;
 
                 @Override
                 public void open() throws HyracksDataException {
-                    if (memsize <= 2) { //Dedicated buffers: One buffer to read and one buffer for output
+                    if (frameLimit <= 2) { //Dedicated buffers: One buffer to read and one buffer for output
                         throw new HyracksDataException("not enough memory for Hybrid Hash Join");
                     }
-                    state.memForJoin = memsize - 2;
+                    state.memForJoin = frameLimit - 2;
                     state.numOfPartitions = getNumberOfPartitions(state.memForJoin, inputsize0, fudgeFactor,
                             nPartitions);
-                    if (!isLeftOuter) {
-                        state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
-                                PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
-                                buildHpc, predEvaluator);
-                    } else {
-                        state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
-                                PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
-                                buildHpc, predEvaluator, isLeftOuter, nullWriterFactories1);
-                    }
+                    state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
+                            PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
+                            buildHpc, predEvaluator, isLeftOuter, nullWriterFactories1);
 
                     state.hybridHJ.initBuild();
-                    LOGGER.fine("OptimizedHybridHashJoin is starting the build phase with " + state.numOfPartitions
-                            + " partitions using " + state.memForJoin + " frames for memory.");
+                    if (LOGGER.isLoggable(Level.FINE)) {
+                        LOGGER.fine("OptimizedHybridHashJoin is starting the build phase with " + state.numOfPartitions
+                                + " partitions using " + state.memForJoin + " frames for memory.");
+                    }
                 }
 
                 @Override
@@ -313,12 +301,19 @@
                 @Override
                 public void close() throws HyracksDataException {
                     state.hybridHJ.closeBuild();
-                    ctx.setStateObject(state);
-                    LOGGER.fine("OptimizedHybridHashJoin closed its build phase");
+                    if (isFailed){
+                        state.hybridHJ.clearBuildTempFiles();
+                    } else {
+                        ctx.setStateObject(state);
+                        if (LOGGER.isLoggable(Level.FINE)) {
+                            LOGGER.fine("OptimizedHybridHashJoin closed its build phase");
+                        }
+                    }
                 }
 
                 @Override
                 public void fail() throws HyracksDataException {
+                    isFailed = true;
                 }
 
             };
@@ -350,22 +345,30 @@
                 IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
                         throws HyracksDataException {
 
-            final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
-            final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
+            final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
             final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
-            final ITuplePairComparator nljComparator0 = tuplePairComparatorFactory0.createTuplePairComparator(ctx);
-            final ITuplePairComparator nljComparator1 = tuplePairComparatorFactory1.createTuplePairComparator(ctx);
-            final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ? null
-                    : predEvaluatorFactory.createPredicateEvaluator());
+            final ITuplePairComparator nljComparatorProbe2Build = tuplePairComparatorFactoryProbe2Build
+                    .createTuplePairComparator(ctx);
+            final ITuplePairComparator nljComparatorBuild2Probe = tuplePairComparatorFactoryBuild2Probe
+                    .createTuplePairComparator(ctx);
+            final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ? null : predEvaluatorFactory
+                    .createPredicateEvaluator());
 
             for (int i = 0; i < comparatorFactories.length; i++) {
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
             }
 
             final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
+            final ArrayTupleBuilder nullTupleBuild = isLeftOuter ?
+                    new ArrayTupleBuilder(buildRd.getFieldCount()) :
+                    null;
             if (isLeftOuter) {
+                DataOutput out = nullTupleBuild.getDataOutput();
                 for (int i = 0; i < nullWriterFactories1.length; i++) {
                     nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
+                    nullWriters1[i].writeNull(out);
+                    nullTupleBuild.addFieldEndOffset();
                 }
             }
 
@@ -373,14 +376,20 @@
                 private BuildAndPartitionTaskState state;
                 private IFrame rPartbuff = new VSizeFrame(ctx);
 
+                private FrameTupleAppender nullResultAppender = null;
+                private FrameTupleAccessor probeTupleAccessor;
+
                 @Override
                 public void open() throws HyracksDataException {
-                    writer.open();
-                    state = (BuildAndPartitionTaskState) ctx.getStateObject(
-                            new TaskId(new ActivityId(getOperatorId(), BUILD_AND_PARTITION_ACTIVITY_ID), partition));
+                    state = (BuildAndPartitionTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
+                            BUILD_AND_PARTITION_ACTIVITY_ID), partition));
 
+                    writer.open();
                     state.hybridHJ.initProbe();
-                    LOGGER.fine("OptimizedHybridHashJoin is starting the probe phase.");
+
+                    if (LOGGER.isLoggable(Level.FINE)) {
+                        LOGGER.fine("OptimizedHybridHashJoin is starting the probe phase.");
+                    }
                 }
 
                 @Override
@@ -397,239 +406,260 @@
                 public void close() throws HyracksDataException {
                     try {
                         state.hybridHJ.closeProbe(writer);
+
                         BitSet partitionStatus = state.hybridHJ.getPartitionStatus();
+
                         rPartbuff.reset();
-                        for (int pid = partitionStatus.nextSetBit(0); pid >= 0; pid = partitionStatus
-                                .nextSetBit(pid + 1)) {
+                        for (int pid = partitionStatus.nextSetBit(0); pid >= 0; pid = partitionStatus.nextSetBit(pid + 1)) {
+
                             RunFileReader bReader = state.hybridHJ.getBuildRFReader(pid);
                             RunFileReader pReader = state.hybridHJ.getProbeRFReader(pid);
-                            if (bReader == null || pReader == null) { //either of sides (or both) does not have any tuple, thus no need for joining (no potential match)
+
+                            if (bReader == null || pReader == null) {
+                                if (isLeftOuter && pReader != null) {
+                                    appendNullToProbeTuples(pReader);
+                                }
                                 continue;
                             }
                             int bSize = state.hybridHJ.getBuildPartitionSizeInTup(pid);
                             int pSize = state.hybridHJ.getProbePartitionSizeInTup(pid);
-                            int beforeMax = (bSize > pSize) ? bSize : pSize;
-                            joinPartitionPair(state.hybridHJ, bReader, pReader, pid, beforeMax, 1, false);
+                            joinPartitionPair(bReader, pReader, bSize, pSize, 1);
                         }
                     } finally {
                         writer.close();
                     }
-                    LOGGER.fine("OptimizedHybridHashJoin closed its probe phase");
+                    if (LOGGER.isLoggable(Level.FINE)) {
+                        LOGGER.fine("OptimizedHybridHashJoin closed its probe phase");
+                    }
                 }
 
-                private void joinPartitionPair(OptimizedHybridHashJoin ohhj, RunFileReader buildSideReader,
-                        RunFileReader probeSideReader, int pid, int beforeMax, int level, boolean wasReversed)
-                                throws HyracksDataException {
+                //The buildSideReader should be always the original buildSideReader, so should the probeSideReader
+                private void joinPartitionPair(RunFileReader buildSideReader, RunFileReader probeSideReader,
+                        int buildSizeInTuple, int probeSizeInTuple, int level)
+                        throws HyracksDataException {
                     ITuplePartitionComputer probeHpc = new FieldHashPartitionComputerFamily(probeKeys,
                             hashFunctionGeneratorFactories).createPartitioner(level);
                     ITuplePartitionComputer buildHpc = new FieldHashPartitionComputerFamily(buildKeys,
                             hashFunctionGeneratorFactories).createPartitioner(level);
 
-                    long buildPartSize = wasReversed ? (ohhj.getProbePartitionSize(pid) / ctx.getInitialFrameSize())
-                            : (ohhj.getBuildPartitionSize(pid) / ctx.getInitialFrameSize());
-                    long probePartSize = wasReversed ? (ohhj.getBuildPartitionSize(pid) / ctx.getInitialFrameSize())
-                            : (ohhj.getProbePartitionSize(pid) / ctx.getInitialFrameSize());
+                    long buildPartSize = buildSideReader.getFileSize() / ctx.getInitialFrameSize();
+                    long probePartSize = probeSideReader.getFileSize() / ctx.getInitialFrameSize();
+                    int beforeMax = Math.max(buildSizeInTuple, probeSizeInTuple);
 
-                    LOGGER.fine("\n>>>Joining Partition Pairs (thread_id " + Thread.currentThread().getId() + ") (pid "
-                            + pid + ") - (level " + level + ") - wasReversed " + wasReversed + " - BuildSize:\t"
-                            + buildPartSize + "\tProbeSize:\t" + probePartSize + " - MemForJoin " + (state.memForJoin)
-                            + "  - LeftOuter is " + isLeftOuter);
+                    if (LOGGER.isLoggable(Level.FINE)) {
+                        LOGGER.fine(
+                                "\n>>>Joining Partition Pairs (thread_id " + Thread.currentThread().getId() + ") (pid "
+                                        + ") - (level " + level + ")"
+                                        + " - BuildSize:\t" + buildPartSize + "\tProbeSize:\t" + probePartSize
+                                        + " - MemForJoin "
+                                        + (state.memForJoin)
+                                        + "  - LeftOuter is " + isLeftOuter);
+                    }
 
                     //Apply in-Mem HJ if possible
-                    if (!skipInMemoryHJ && (buildPartSize < state.memForJoin)
-                            || (probePartSize < state.memForJoin && !isLeftOuter)) {
+                    if (!skipInMemoryHJ && ((buildPartSize < state.memForJoin)
+                            || (probePartSize < state.memForJoin && !isLeftOuter))) {
                         int tabSize = -1;
-                        if (!forceRR && (isLeftOuter || (buildPartSize < probePartSize))) { //Case 1.1 - InMemHJ (wout Role-Reversal)
-                            LOGGER.fine("\t>>>Case 1.1 (IsLeftOuter || buildSize<probe) AND ApplyInMemHJ - [Level "
-                                    + level + "]");
-                            tabSize = wasReversed ? ohhj.getProbePartitionSizeInTup(pid)
-                                    : ohhj.getBuildPartitionSizeInTup(pid);
+                        if (!forceRR && (isLeftOuter || (buildPartSize
+                                < probePartSize))) { //Case 1.1 - InMemHJ (wout Role-Reversal)
+                            if (LOGGER.isLoggable(Level.FINE)) {
+                                LOGGER.fine("\t>>>Case 1.1 (IsLeftOuter || buildSize<probe) AND ApplyInMemHJ - [Level "
+                                        + level + "]");
+                            }
+                            tabSize = buildSizeInTuple;
                             if (tabSize == 0) {
                                 throw new HyracksDataException(
                                         "Trying to join an empty partition. Invalid table size for inMemoryHashJoin.");
                             }
                             //Build Side is smaller
-                            applyInMemHashJoin(buildKeys, probeKeys, tabSize, probeRd, buildRd, probeHpc, buildHpc,
-                                    buildSideReader, probeSideReader, false, pid); // checked-confirmed
+                            applyInMemHashJoin(buildKeys, probeKeys, tabSize, buildRd, probeRd, buildHpc,
+                                    probeHpc, buildSideReader, probeSideReader); // checked-confirmed
                         } else { //Case 1.2 - InMemHJ with Role Reversal
-                            LOGGER.fine(
-                                    "\t>>>Case 1.2. (NoIsLeftOuter || probe<build) AND ApplyInMemHJ WITH RoleReversal - [Level "
-                                            + level + "]");
-                            tabSize = wasReversed ? ohhj.getBuildPartitionSizeInTup(pid)
-                                    : ohhj.getProbePartitionSizeInTup(pid);
+                            if (LOGGER.isLoggable(Level.FINE)) {
+                                LOGGER.fine(
+                                        "\t>>>Case 1.2. (NoIsLeftOuter || probe<build) AND ApplyInMemHJ WITH RoleReversal - [Level "
+                                                + level + "]");
+                            }
+                            tabSize = probeSizeInTuple;
                             if (tabSize == 0) {
                                 throw new HyracksDataException(
                                         "Trying to join an empty partition. Invalid table size for inMemoryHashJoin.");
                             }
                             //Probe Side is smaller
-                            applyInMemHashJoin(probeKeys, buildKeys, tabSize, buildRd, probeRd, buildHpc, probeHpc,
-                                    probeSideReader, buildSideReader, true, pid); // checked-confirmed
+                            applyInMemHashJoin(probeKeys, buildKeys, tabSize, probeRd, buildRd, probeHpc,
+                                    buildHpc, probeSideReader, buildSideReader); // checked-confirmed
                         }
                     }
                     //Apply (Recursive) HHJ
                     else {
-                        LOGGER.fine("\t>>>Case 2. ApplyRecursiveHHJ - [Level " + level + "]");
-                        OptimizedHybridHashJoin rHHj;
-                        if (!forceRR && (isLeftOuter || buildPartSize < probePartSize)) { //Case 2.1 - Recursive HHJ (wout Role-Reversal)
-                            LOGGER.fine("\t\t>>>Case 2.1 - RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
-                                    + level + "]");
-                            int n = getNumberOfPartitions(state.memForJoin, (int) buildPartSize, fudgeFactor,
-                                    nPartitions);
-                            rHHj = new OptimizedHybridHashJoin(ctx, state.memForJoin, n, PROBE_REL, BUILD_REL,
-                                    probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc, buildHpc,
-                                    predEvaluator); //checked-confirmed
-
-                            buildSideReader.open();
-                            rHHj.initBuild();
-                            rPartbuff.reset();
-                            while (buildSideReader.nextFrame(rPartbuff)) {
-                                rHHj.build(rPartbuff.getBuffer());
-                            }
-
-                            rHHj.closeBuild();
-
-                            probeSideReader.open();
-                            rHHj.initProbe();
-                            rPartbuff.reset();
-                            while (probeSideReader.nextFrame(rPartbuff)) {
-                                rHHj.probe(rPartbuff.getBuffer(), writer);
-                            }
-                            rHHj.closeProbe(writer);
-
-                            int maxAfterBuildSize = rHHj.getMaxBuildPartitionSize();
-                            int maxAfterProbeSize = rHHj.getMaxProbePartitionSize();
-                            int afterMax = (maxAfterBuildSize > maxAfterProbeSize) ? maxAfterBuildSize
-                                    : maxAfterProbeSize;
-
-                            BitSet rPStatus = rHHj.getPartitionStatus();
-                            if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD * beforeMax))) { //Case 2.1.1 - Keep applying HHJ
-                                LOGGER.fine(
-                                        "\t\t>>>Case 2.1.1 - KEEP APPLYING RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
-                                                + level + "]");
-                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus
-                                        .nextSetBit(rPid + 1)) {
-                                    RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
-                                    RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
-
-                                    if (rbrfw == null || rprfw == null) {
-                                        continue;
-                                    }
-
-                                    joinPartitionPair(rHHj, rbrfw, rprfw, rPid, afterMax, (level + 1), false); // checked-confirmed
-                                }
-
-                            } else { //Case 2.1.2 - Switch to NLJ
-                                LOGGER.fine(
-                                        "\t\t>>>Case 2.1.2 - SWITCHED to NLJ RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
-                                                + level + "]");
-                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus
-                                        .nextSetBit(rPid + 1)) {
-                                    RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
-                                    RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
-
-                                    if (rbrfw == null || rprfw == null) {
-                                        continue;
-                                    }
-
-                                    int buildSideInTups = rHHj.getBuildPartitionSizeInTup(rPid);
-                                    int probeSideInTups = rHHj.getProbePartitionSizeInTup(rPid);
-                                    if (isLeftOuter || buildSideInTups < probeSideInTups) {
-                                        applyNestedLoopJoin(buildRd, probeRd, memsize, rprfw, rbrfw, nljComparator0,
-                                                false); //checked-modified
-                                    } else {
-                                        applyNestedLoopJoin(probeRd, buildRd, memsize, rbrfw, rprfw, nljComparator1,
-                                                true); //checked-modified
-                                    }
-                                }
-                            }
-                        } else { //Case 2.2 - Recursive HHJ (with Role-Reversal)
-                            LOGGER.fine("\t\t>>>Case 2.2. - RecursiveHHJ WITH RoleReversal - [Level " + level + "]");
-                            int n = getNumberOfPartitions(state.memForJoin, (int) probePartSize, fudgeFactor,
-                                    nPartitions);
-
-                            rHHj = new OptimizedHybridHashJoin(ctx, state.memForJoin, n, BUILD_REL, PROBE_REL,
-                                    buildKeys, probeKeys, comparators, buildRd, probeRd, buildHpc, probeHpc,
-                                    predEvaluator); //checked-confirmed
-                            rHHj.setIsReversed(true); //Added to use predicateEvaluator (for inMemoryHashJoin) correctly
-
-                            probeSideReader.open();
-                            rHHj.initBuild();
-                            rPartbuff.reset();
-                            while (probeSideReader.nextFrame(rPartbuff)) {
-                                rHHj.build(rPartbuff.getBuffer());
-                            }
-                            rHHj.closeBuild();
-                            rHHj.initProbe();
-                            buildSideReader.open();
-                            rPartbuff.reset();
-                            while (buildSideReader.nextFrame(rPartbuff)) {
-                                rHHj.probe(rPartbuff.getBuffer(), writer);
-                            }
-                            rHHj.closeProbe(writer);
-                            int maxAfterBuildSize = rHHj.getMaxBuildPartitionSize();
-                            int maxAfterProbeSize = rHHj.getMaxProbePartitionSize();
-                            int afterMax = (maxAfterBuildSize > maxAfterProbeSize) ? maxAfterBuildSize
-                                    : maxAfterProbeSize;
-                            BitSet rPStatus = rHHj.getPartitionStatus();
-
-                            if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD * beforeMax))) { //Case 2.2.1 - Keep applying HHJ
-                                LOGGER.fine("\t\t>>>Case 2.2.1 - KEEP APPLYING RecursiveHHJ WITH RoleReversal - [Level "
-                                        + level + "]");
-                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus
-                                        .nextSetBit(rPid + 1)) {
-                                    RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
-                                    RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
-
-                                    if (rbrfw == null || rprfw == null) {
-                                        continue;
-                                    }
-
-                                    joinPartitionPair(rHHj, rprfw, rbrfw, rPid, afterMax, (level + 1), true); //checked-confirmed
-                                }
-                            } else { //Case 2.2.2 - Switch to NLJ
-                                LOGGER.fine(
-                                        "\t\t>>>Case 2.2.2 - SWITCHED to NLJ RecursiveHHJ WITH RoleReversal - [Level "
-                                                + level + "]");
-                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus
-                                        .nextSetBit(rPid + 1)) {
-                                    RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
-                                    RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
-
-                                    if (rbrfw == null || rprfw == null) {
-                                        continue;
-                                    }
-
-                                    long buildSideSize = rbrfw.getFileSize();
-                                    long probeSideSize = rprfw.getFileSize();
-                                    if (buildSideSize > probeSideSize) {
-                                        applyNestedLoopJoin(buildRd, probeRd, memsize, rbrfw, rprfw, nljComparator0,
-                                                true); //checked-modified
-                                    } else {
-                                        applyNestedLoopJoin(probeRd, buildRd, memsize, rprfw, rbrfw, nljComparator1,
-                                                true); //checked-modified
-                                    }
-                                }
-                            }
+                        if (LOGGER.isLoggable(Level.FINE)) {
+                            LOGGER.fine("\t>>>Case 2. ApplyRecursiveHHJ - [Level " + level + "]");
                         }
-                        buildSideReader.close();
-                        probeSideReader.close();
+                        if (!forceRR && (isLeftOuter
+                                || buildPartSize < probePartSize)) { //Case 2.1 - Recursive HHJ (wout Role-Reversal)
+                            if (LOGGER.isLoggable(Level.FINE)) {
+                                LOGGER.fine("\t\t>>>Case 2.1 - RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+                                        + level + "]");
+                            }
+                            applyHybridHashJoin((int) buildPartSize, PROBE_REL, BUILD_REL, probeKeys, buildKeys,
+                                    probeRd, buildRd, probeHpc, buildHpc, probeSideReader, buildSideReader, level,
+                                    beforeMax);
+
+                        } else { //Case 2.2 - Recursive HHJ (with Role-Reversal)
+                            if (LOGGER.isLoggable(Level.FINE)) {
+                                LOGGER.fine(
+                                        "\t\t>>>Case 2.2. - RecursiveHHJ WITH RoleReversal - [Level " + level + "]");
+                            }
+
+                            applyHybridHashJoin((int) probePartSize, BUILD_REL, PROBE_REL, buildKeys, probeKeys,
+                                    buildRd, probeRd, buildHpc, probeHpc, buildSideReader, probeSideReader, level,
+                                    beforeMax);
+
+                        }
                     }
                 }
 
+                private void applyHybridHashJoin(int tableSize,
+                        final String PROBE_REL, final String BUILD_REL,
+                        final int[] probeKeys, final int[] buildKeys,
+                        final RecordDescriptor probeRd, final RecordDescriptor buildRd,
+                        final ITuplePartitionComputer probeHpc, final ITuplePartitionComputer buildHpc,
+                        RunFileReader probeSideReader, RunFileReader buildSideReader,
+                        final int level, final long beforeMax)
+                        throws HyracksDataException {
+
+                    boolean isReversed = probeKeys == OptimizedHybridHashJoinOperatorDescriptor.this.buildKeys
+                            && buildKeys == OptimizedHybridHashJoinOperatorDescriptor.this.probeKeys;
+
+                    assert isLeftOuter ? !isReversed : true : "LeftOut Join can not reverse roles";
+
+                    OptimizedHybridHashJoin rHHj;
+                    int n = getNumberOfPartitions(state.memForJoin, tableSize, fudgeFactor,
+                            nPartitions);
+                    rHHj = new OptimizedHybridHashJoin(ctx, state.memForJoin, n, PROBE_REL, BUILD_REL,
+                            probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc, buildHpc,
+                            predEvaluator, isLeftOuter, nullWriterFactories1); //checked-confirmed
+
+                    rHHj.setIsReversed(isReversed);
+                    buildSideReader.open();
+                    rHHj.initBuild();
+                    rPartbuff.reset();
+                    while (buildSideReader.nextFrame(rPartbuff)) {
+                        rHHj.build(rPartbuff.getBuffer());
+                    }
+                    rHHj.closeBuild();
+                    buildSideReader.close();
+
+                    probeSideReader.open();
+                    rHHj.initProbe();
+                    rPartbuff.reset();
+                    while (probeSideReader.nextFrame(rPartbuff)) {
+                        rHHj.probe(rPartbuff.getBuffer(), writer);
+                    }
+                    rHHj.closeProbe(writer);
+                    probeSideReader.close();
+
+                    int maxAfterBuildSize = rHHj.getMaxBuildPartitionSize();
+                    int maxAfterProbeSize = rHHj.getMaxProbePartitionSize();
+                    int afterMax = Math.max(maxAfterBuildSize, maxAfterProbeSize);
+
+                    BitSet rPStatus = rHHj.getPartitionStatus();
+                    if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD
+                            * beforeMax))) { //Case 2.1.1 - Keep applying HHJ
+                        if (LOGGER.isLoggable(Level.FINE)) {
+                            LOGGER.fine(
+                                    "\t\t>>>Case 2.1.1 - KEEP APPLYING RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+                                            + level + "]");
+                        }
+                        for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+                            RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
+                            RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
+                            int rbSizeInTuple = rHHj.getBuildPartitionSizeInTup(rPid);
+                            int rpSizeInTuple = rHHj.getProbePartitionSizeInTup(rPid);
+
+                            if (rbrfw == null || rprfw == null) {
+                                if (isLeftOuter && rprfw != null) { // if outer join, we don't reverse
+                                    appendNullToProbeTuples(rprfw);
+                                }
+                                continue;
+                            }
+
+                            if (isReversed) {
+                                joinPartitionPair(rprfw, rbrfw, rpSizeInTuple, rbSizeInTuple, level + 1);
+                            } else {
+                                joinPartitionPair(rbrfw, rprfw, rbSizeInTuple, rpSizeInTuple, level + 1);
+                            }
+                        }
+
+                    } else { //Case 2.1.2 - Switch to NLJ
+                        if (LOGGER.isLoggable(Level.FINE)) {
+                            LOGGER.fine(
+                                    "\t\t>>>Case 2.1.2 - SWITCHED to NLJ RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+                                            + level + "]");
+                        }
+                        for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+                            RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
+                            RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
+
+                            if (rbrfw == null || rprfw == null) {
+                                if (isLeftOuter && rprfw != null) { // if outer join, we don't reverse
+                                    appendNullToProbeTuples(rprfw);
+                                }
+                                continue;
+                            }
+
+                            int buildSideInTups = rHHj.getBuildPartitionSizeInTup(rPid);
+                            int probeSideInTups = rHHj.getProbePartitionSizeInTup(rPid);
+                            // NLJ order is outer + inner, the order is reversed from the other joins
+                            if (isLeftOuter || probeSideInTups < buildSideInTups) {
+                                applyNestedLoopJoin(probeRd, buildRd, frameLimit, rprfw, rbrfw); //checked-modified
+                            } else {
+                                applyNestedLoopJoin(buildRd, probeRd, frameLimit, rbrfw, rprfw); //checked-modified
+                            }
+                        }
+                    }
+                }
+
+                private void appendNullToProbeTuples(RunFileReader probReader) throws HyracksDataException {
+                    if (nullResultAppender == null) {
+                        nullResultAppender = new FrameTupleAppender(new VSizeFrame(ctx));
+                    }
+                    if (probeTupleAccessor == null) {
+                        probeTupleAccessor = new FrameTupleAccessor(probeRd);
+                    }
+                    probReader.open();
+                    while (probReader.nextFrame(rPartbuff)) {
+                        probeTupleAccessor.reset(rPartbuff.getBuffer());
+                        for (int tid = 0; tid < probeTupleAccessor.getTupleCount(); tid++) {
+                            FrameUtils.appendConcatToWriter(writer, nullResultAppender, probeTupleAccessor, tid,
+                                    nullTupleBuild.getFieldEndOffsets(), nullTupleBuild.getByteArray(), 0,
+                                    nullTupleBuild.getSize());
+                        }
+                    }
+                    probReader.close();
+                    nullResultAppender.write(writer, true);
+                }
+
                 private void applyInMemHashJoin(int[] bKeys, int[] pKeys, int tabSize, RecordDescriptor buildRDesc,
-                        RecordDescriptor probeRDesc, ITuplePartitionComputer hpcRepLarger,
-                        ITuplePartitionComputer hpcRepSmaller, RunFileReader bReader, RunFileReader pReader,
-                        boolean reverse, int pid) throws HyracksDataException {
+                        RecordDescriptor probeRDesc, ITuplePartitionComputer hpcRepBuild,
+                        ITuplePartitionComputer hpcRepProbe, RunFileReader bReader, RunFileReader pReader)
+                        throws HyracksDataException {
+                    boolean isReversed = pKeys == OptimizedHybridHashJoinOperatorDescriptor.this.buildKeys
+                            && bKeys == OptimizedHybridHashJoinOperatorDescriptor.this.probeKeys;
+
+                    assert isLeftOuter ? !isReversed : true : "LeftOut Join can not reverse roles";
+
                     ISerializableTable table = new SerializableHashTable(tabSize, ctx);
                     InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tabSize, new FrameTupleAccessor(probeRDesc),
-                            hpcRepLarger, new FrameTupleAccessor(buildRDesc), hpcRepSmaller,
+                            hpcRepProbe, new FrameTupleAccessor(buildRDesc), hpcRepBuild,
                             new FrameTuplePairComparator(pKeys, bKeys, comparators), isLeftOuter, nullWriters1, table,
-                            predEvaluator, reverse);
+                            predEvaluator, isReversed);
 
                     bReader.open();
                     rPartbuff.reset();
                     while (bReader.nextFrame(rPartbuff)) {
-                        ByteBuffer copyBuffer = ctx.allocateFrame(rPartbuff.getFrameSize()); //We need to allocate a copyBuffer, because this buffer gets added to the buffers list in the InMemoryHashJoin
+                        //We need to allocate a copyBuffer, because this buffer gets added to the buffers list in the InMemoryHashJoin
+                        ByteBuffer copyBuffer = ctx.allocateFrame(rPartbuff.getFrameSize());
                         FrameUtils.copyAndFlip(rPartbuff.getBuffer(), copyBuffer);
                         joiner.build(copyBuffer);
                         rPartbuff.reset();
@@ -647,12 +677,19 @@
                 }
 
                 private void applyNestedLoopJoin(RecordDescriptor outerRd, RecordDescriptor innerRd, int memorySize,
-                        RunFileReader outerReader, RunFileReader innerReader, ITuplePairComparator nljComparator,
-                        boolean reverse) throws HyracksDataException {
-                    NestedLoopJoin nlj = new NestedLoopJoin(ctx, new FrameTupleAccessor(outerRd),
-                            new FrameTupleAccessor(innerRd), nljComparator, memorySize, predEvaluator, isLeftOuter,
-                            nullWriters1);
-                    nlj.setIsReversed(reverse);
+                        RunFileReader outerReader, RunFileReader innerReader)
+                        throws HyracksDataException {
+                    // The nested loop join result is outer + inner. All the other operator is probe + build. Hence the reverse relation is different
+                    boolean isReversed = outerRd == buildRd && innerRd == probeRd;
+                    assert isLeftOuter ? !isReversed : true : "LeftOut Join can not reverse roles";
+                    ITuplePairComparator nljComptorOuterInner = isReversed ?
+                            nljComparatorBuild2Probe :
+                            nljComparatorProbe2Build;
+                    NestedLoopJoin nlj = new NestedLoopJoin(ctx,
+                            new FrameTupleAccessor(outerRd),
+                            new FrameTupleAccessor(innerRd), nljComptorOuterInner, memorySize,
+                            predEvaluator, isLeftOuter, nullWriters1);
+                    nlj.setIsReversed(isReversed);
 
                     IFrame cacheBuff = new VSizeFrame(ctx);
                     innerReader.open();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunGenerator.java
new file mode 100644
index 0000000..7c7bfec
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunGenerator.java
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
+import org.apache.hyracks.dataflow.std.buffermanager.FrameFreeSlotPolicyFactory;
+import org.apache.hyracks.dataflow.std.buffermanager.IFrameBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.IFrameFreeSlotPolicy;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFrameMemoryManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFramePool;
+
+public abstract class AbstractExternalSortRunGenerator extends AbstractSortRunGenerator {
+
+    protected final IHyracksTaskContext ctx;
+    protected final IFrameSorter frameSorter;
+    protected final int maxSortFrames;
+
+    public AbstractExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDesc, Algorithm alg, int framesLimit) throws HyracksDataException {
+        this(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc, alg,
+                EnumFreeSlotPolicy.LAST_FIT, framesLimit);
+    }
+
+    public AbstractExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDesc, Algorithm alg, EnumFreeSlotPolicy policy, int framesLimit)
+                    throws HyracksDataException {
+        this(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc, alg, policy, framesLimit,
+                Integer.MAX_VALUE);
+    }
+
+    public AbstractExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDesc, Algorithm alg, EnumFreeSlotPolicy policy, int framesLimit, int outputLimit)
+                    throws HyracksDataException {
+        super();
+        this.ctx = ctx;
+        maxSortFrames = framesLimit - 1;
+
+        IFrameFreeSlotPolicy freeSlotPolicy = FrameFreeSlotPolicyFactory.createFreeSlotPolicy(policy, maxSortFrames);
+        IFrameBufferManager bufferManager = new VariableFrameMemoryManager(
+                new VariableFramePool(ctx, maxSortFrames * ctx.getInitialFrameSize()), freeSlotPolicy);
+        if (alg == Algorithm.MERGE_SORT) {
+            frameSorter = new FrameSorterMergeSort(ctx, bufferManager, sortFields, firstKeyNormalizerFactory,
+                    comparatorFactories, recordDesc, outputLimit);
+        } else {
+            frameSorter = new FrameSorterQuickSort(ctx, bufferManager, sortFields, firstKeyNormalizerFactory,
+                    comparatorFactories, recordDesc, outputLimit);
+        }
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        if (!frameSorter.insertFrame(buffer)) {
+            flushFramesToRun();
+            if (!frameSorter.insertFrame(buffer)) {
+                throw new HyracksDataException("The given frame is too big to insert into the sorting memory.");
+            }
+        }
+    }
+
+    @Override
+    public ISorter getSorter() {
+        return frameSorter;
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java
new file mode 100644
index 0000000..24154f7
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java
@@ -0,0 +1,243 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.sort;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.hyracks.api.comm.FrameConstants;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.sort.util.GroupVSizeFrame;
+
+public abstract class AbstractExternalSortRunMerger {
+
+    protected final IHyracksTaskContext ctx;
+    protected final IFrameWriter writer;
+
+    private final List<GeneratedRunFileReader> runs;
+    private final BitSet currentGenerationRunAvailable;
+    private final IBinaryComparator[] comparators;
+    private final INormalizedKeyComputer nmkComputer;
+    private final RecordDescriptor recordDesc;
+    private final int framesLimit;
+    private final int MAX_FRAME_SIZE;
+    private final int topK;
+    private List<GroupVSizeFrame> inFrames;
+    private VSizeFrame outputFrame;
+    private ISorter sorter;
+
+    private static final Logger LOGGER = Logger.getLogger(AbstractExternalSortRunMerger.class.getName());
+
+    public AbstractExternalSortRunMerger(IHyracksTaskContext ctx, ISorter sorter, List<GeneratedRunFileReader> runs,
+            IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc,
+            int framesLimit, IFrameWriter writer) {
+        this(ctx, sorter, runs, comparators, nmkComputer, recordDesc, framesLimit, Integer.MAX_VALUE, writer);
+    }
+
+    public AbstractExternalSortRunMerger(IHyracksTaskContext ctx, ISorter sorter, List<GeneratedRunFileReader> runs,
+            IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc,
+            int framesLimit, int topK, IFrameWriter writer) {
+        this.ctx = ctx;
+        this.sorter = sorter;
+        this.runs = new LinkedList<>(runs);
+        this.currentGenerationRunAvailable = new BitSet(runs.size());
+        this.comparators = comparators;
+        this.nmkComputer = nmkComputer;
+        this.recordDesc = recordDesc;
+        this.framesLimit = framesLimit;
+        this.writer = writer;
+        this.MAX_FRAME_SIZE = FrameConstants.MAX_NUM_MINFRAME * ctx.getInitialFrameSize();
+        this.topK = topK;
+    }
+
+    public void process() throws HyracksDataException {
+        IFrameWriter finalWriter = null;
+        try {
+            if (runs.size() <= 0) {
+                finalWriter = prepareSkipMergingFinalResultWriter(writer);
+                finalWriter.open();
+                if (sorter != null) {
+                    if (sorter.hasRemaining()) {
+                        sorter.flush(finalWriter);
+                    }
+                    sorter.close();
+                }
+            } else {
+                /** recycle sort buffer */
+                if (sorter != null) {
+                    sorter.close();
+                }
+
+                finalWriter = prepareFinalMergeResultWriter(writer);
+                finalWriter.open();
+
+                int maxMergeWidth = framesLimit - 1;
+
+                inFrames = new ArrayList<>(maxMergeWidth);
+                outputFrame = new VSizeFrame(ctx);
+                List<GeneratedRunFileReader> partialRuns = new ArrayList<>(maxMergeWidth);
+
+                int stop = runs.size();
+                currentGenerationRunAvailable.set(0, stop);
+
+                while (true) {
+
+                    int unUsed = selectPartialRuns(maxMergeWidth * ctx.getInitialFrameSize(), runs, partialRuns,
+                            currentGenerationRunAvailable, stop);
+                    prepareFrames(unUsed, inFrames, partialRuns);
+
+                    if (!currentGenerationRunAvailable.isEmpty() || stop < runs.size()) {
+                        GeneratedRunFileReader reader;
+                        if (partialRuns.size() == 1) {
+                            if (!currentGenerationRunAvailable.isEmpty()) {
+                                throw new HyracksDataException(
+                                        "The record is too big to put into the merging frame, please"
+                                                + " allocate more sorting memory");
+                            } else {
+                                reader = partialRuns.get(0);
+                            }
+
+                        } else {
+                            RunFileWriter mergeFileWriter = prepareIntermediateMergeRunFile();
+                            IFrameWriter mergeResultWriter = prepareIntermediateMergeResultWriter(mergeFileWriter);
+
+                            mergeResultWriter.open();
+                            merge(mergeResultWriter, partialRuns);
+                            mergeResultWriter.close();
+
+                            reader = mergeFileWriter.createReader();
+                        }
+                        runs.add(reader);
+
+                        if (currentGenerationRunAvailable.isEmpty()) {
+
+                            if (LOGGER.isLoggable(Level.FINE)) {
+                                LOGGER.fine("generated runs:" + stop);
+                            }
+                            runs.subList(0, stop).clear();
+                            currentGenerationRunAvailable.clear();
+                            currentGenerationRunAvailable.set(0, runs.size());
+                            stop = runs.size();
+                        }
+                    } else {
+                        if (LOGGER.isLoggable(Level.FINE)) {
+                            LOGGER.fine("final runs:" + stop);
+                        }
+                        merge(finalWriter, partialRuns);
+                        break;
+                    }
+                }
+            }
+        } catch (Exception e) {
+            finalWriter.fail();
+            throw new HyracksDataException(e);
+        } finally {
+            finalWriter.close();
+        }
+    }
+
+    private static int selectPartialRuns(int budget, List<GeneratedRunFileReader> runs,
+            List<GeneratedRunFileReader> partialRuns, BitSet runAvailable, int stop) {
+        partialRuns.clear();
+        int maxFrameSizeOfGenRun = 0;
+        int nextRunId = runAvailable.nextSetBit(0);
+        while (budget > 0 && nextRunId >= 0 && nextRunId < stop) {
+            int runFrameSize = runs.get(nextRunId).getMaxFrameSize();
+            if (budget - runFrameSize >= 0) {
+                partialRuns.add(runs.get(nextRunId));
+                budget -= runFrameSize;
+                runAvailable.clear(nextRunId);
+                maxFrameSizeOfGenRun = Math.max(runFrameSize, maxFrameSizeOfGenRun);
+            }
+            nextRunId = runAvailable.nextSetBit(nextRunId + 1);
+        }
+        return budget;
+    }
+
+    private void prepareFrames(int extraFreeMem, List<GroupVSizeFrame> inFrames,
+            List<GeneratedRunFileReader> partialRuns) throws HyracksDataException {
+        if (extraFreeMem > 0 && partialRuns.size() > 1) {
+            int extraFrames = extraFreeMem / ctx.getInitialFrameSize();
+            int avg = (extraFrames / partialRuns.size()) * ctx.getInitialFrameSize();
+            int residue = (extraFrames % partialRuns.size());
+            for (int i = 0; i < residue; i++) {
+                partialRuns.get(i).updateSize(Math.min(MAX_FRAME_SIZE,
+                        partialRuns.get(i).getMaxFrameSize() + avg + ctx.getInitialFrameSize()));
+            }
+            for (int i = residue; i < partialRuns.size() && avg > 0; i++) {
+                partialRuns.get(i).updateSize(Math.min(MAX_FRAME_SIZE, partialRuns.get(i).getMaxFrameSize() + avg));
+            }
+        }
+
+        if (inFrames.size() > partialRuns.size()) {
+            inFrames.subList(partialRuns.size(), inFrames.size()).clear();
+        }
+        int i;
+        for (i = 0; i < inFrames.size(); i++) {
+            inFrames.get(i).resize(partialRuns.get(i).getMaxFrameSize());
+        }
+        for (; i < partialRuns.size(); i++) {
+            inFrames.add(new GroupVSizeFrame(ctx, partialRuns.get(i).getMaxFrameSize()));
+        }
+    }
+
+    abstract protected IFrameWriter prepareSkipMergingFinalResultWriter(IFrameWriter nextWriter)
+            throws HyracksDataException;
+
+    abstract protected RunFileWriter prepareIntermediateMergeRunFile() throws HyracksDataException;
+
+    abstract protected IFrameWriter prepareIntermediateMergeResultWriter(RunFileWriter mergeFileWriter)
+            throws HyracksDataException;
+
+    abstract protected IFrameWriter prepareFinalMergeResultWriter(IFrameWriter nextWriter) throws HyracksDataException;
+
+    abstract protected int[] getSortFields();
+
+    private void merge(IFrameWriter writer, List<GeneratedRunFileReader> partialRuns) throws HyracksDataException {
+        RunMergingFrameReader merger = new RunMergingFrameReader(ctx, partialRuns, inFrames, getSortFields(),
+                comparators, nmkComputer, recordDesc, topK);
+        int io = 0;
+        merger.open();
+        try {
+            while (merger.nextFrame(outputFrame)) {
+                FrameUtils.flushFrame(outputFrame.getBuffer(), writer);
+                io++;
+            }
+        } finally {
+            merger.close();
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Output " + io + " frames");
+            }
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
index e56d51d..f213b97 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
@@ -36,8 +36,10 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.util.IntSerDeUtils;
+import org.apache.hyracks.dataflow.std.buffermanager.BufferInfo;
+import org.apache.hyracks.dataflow.std.buffermanager.IFrameBufferManager;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
 
 public abstract class AbstractFrameSorter implements IFrameSorter {
 
@@ -60,6 +62,9 @@
     protected int[] tPointers;
     protected int tupleCount;
 
+    private FrameTupleAccessor fta2;
+    private BufferInfo info = new BufferInfo(null, -1, -1);
+
     public AbstractFrameSorter(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor) throws HyracksDataException {
@@ -82,6 +87,7 @@
         this.outputAppender = new FrameTupleAppender();
         this.outputFrame = new VSizeFrame(ctx);
         this.outputLimit = outputLimit;
+        this.fta2 = new FrameTupleAccessor(recordDescriptor);
     }
 
     @Override
@@ -106,9 +112,8 @@
     public void sort() throws HyracksDataException {
         tupleCount = 0;
         for (int i = 0; i < bufferManager.getNumFrames(); ++i) {
-            inputTupleAccessor
-                    .reset(bufferManager.getFrame(i), bufferManager.getFrameStartOffset(i),
-                            bufferManager.getFrameSize(i));
+            bufferManager.getFrame(i, info);
+            inputTupleAccessor.reset(info.getBuffer(), info.getStartOffset(), info.getLength());
             tupleCount += inputTupleAccessor.getTupleCount();
         }
         if (tPointers == null || tPointers.length < tupleCount * PTR_SIZE) {
@@ -117,9 +122,8 @@
         int ptr = 0;
         int sfIdx = sortFields[0];
         for (int i = 0; i < bufferManager.getNumFrames(); ++i) {
-            inputTupleAccessor
-                    .reset(bufferManager.getFrame(i), bufferManager.getFrameStartOffset(i),
-                            bufferManager.getFrameSize(i));
+            bufferManager.getFrame(i, info);
+            inputTupleAccessor.reset(info.getBuffer(), info.getStartOffset(), info.getLength());
             int tCount = inputTupleAccessor.getTupleCount();
             byte[] array = inputTupleAccessor.getBuffer().array();
             for (int j = 0; j < tCount; ++j) {
@@ -163,9 +167,8 @@
             int i = tPointers[ptr * PTR_SIZE + ID_FRAMEID];
             int tStart = tPointers[ptr * PTR_SIZE + ID_TUPLE_START];
             int tEnd = tPointers[ptr * PTR_SIZE + ID_TUPLE_END];
-            ByteBuffer buffer = bufferManager.getFrame(i);
-            inputTupleAccessor.reset(buffer, bufferManager.getFrameStartOffset(i), bufferManager.getFrameSize(i));
-
+            bufferManager.getFrame(i, info);
+            inputTupleAccessor.reset(info.getBuffer(), info.getStartOffset(), info.getLength());
             int flushed = FrameUtils.appendToWriter(writer, outputAppender, inputTupleAccessor, tStart, tEnd);
             if (flushed > 0) {
                 maxFrameSize = Math.max(maxFrameSize, flushed);
@@ -181,6 +184,45 @@
         return maxFrameSize;
     }
 
+    protected final int compare(int tp1, int tp2) throws HyracksDataException {
+        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;
+        }
+        int i2 = tp2i;
+        int j2 = tp2j;
+        bufferManager.getFrame(i1, info);
+        byte[] b1 = info.getBuffer().array();
+        inputTupleAccessor.reset(info.getBuffer(), info.getStartOffset(), info.getLength());
+
+        bufferManager.getFrame(i2, info);
+        byte[] b2 = info.getBuffer().array();
+        fta2.reset(info.getBuffer(), info.getStartOffset(), info.getLength());
+        for (int f = 0; f < comparators.length; ++f) {
+            int fIdx = sortFields[f];
+            int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(b1, j1 + (fIdx - 1) * 4);
+            int f1End = IntSerDeUtils.getInt(b1, j1 + fIdx * 4);
+            int s1 = j1 + inputTupleAccessor.getFieldSlotsLength() + f1Start;
+            int l1 = f1End - f1Start;
+            int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(b2, j2 + (fIdx - 1) * 4);
+            int f2End = IntSerDeUtils.getInt(b2, j2 + fIdx * 4);
+            int s2 = j2 + fta2.getFieldSlotsLength() + f2Start;
+            int l2 = f2End - f2Start;
+            int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+
     @Override
     public void close() {
         tupleCount = 0;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java
index b5ba099..bcebb7d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java
@@ -24,26 +24,27 @@
 
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
 
 public abstract class AbstractSortRunGenerator implements IRunGenerator {
-    protected final List<RunAndMaxFrameSizePair> runAndMaxSizes;
+    protected final List<GeneratedRunFileReader> generatedRunFileReaders;
 
     public AbstractSortRunGenerator() {
-        runAndMaxSizes = new LinkedList<>();
+        generatedRunFileReaders = new LinkedList<>();
     }
 
     abstract public ISorter getSorter() throws HyracksDataException;
 
     @Override
     public void open() throws HyracksDataException {
-        runAndMaxSizes.clear();
+        generatedRunFileReaders.clear();
     }
 
     @Override
     public void close() throws HyracksDataException {
         if (getSorter().hasRemaining()) {
-            if (runAndMaxSizes.size() <= 0) {
+            if (generatedRunFileReaders.size() <= 0) {
                 getSorter().sort();
             } else {
                 flushFramesToRun();
@@ -60,13 +61,12 @@
         RunFileWriter runWriter = getRunFileWriter();
         IFrameWriter flushWriter = getFlushableFrameWriter(runWriter);
         flushWriter.open();
-        int maxFlushedFrameSize;
         try {
-            maxFlushedFrameSize = getSorter().flush(flushWriter);
+            getSorter().flush(flushWriter);
         } finally {
             flushWriter.close();
         }
-        runAndMaxSizes.add(new RunAndMaxFrameSizePair(runWriter.createDeleteOnCloseReader(), maxFlushedFrameSize));
+        generatedRunFileReaders.add(runWriter.createDeleteOnCloseReader());
         getSorter().reset();
     }
 
@@ -75,7 +75,7 @@
     }
 
     @Override
-    public List<RunAndMaxFrameSizePair> getRuns() {
-        return runAndMaxSizes;
+    public List<GeneratedRunFileReader> getRuns() {
+        return generatedRunFileReaders;
     }
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
index 8856c41..665a90b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
@@ -42,6 +42,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
 import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
 import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
 import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
@@ -92,7 +93,7 @@
     }
 
     public static class SortTaskState extends AbstractStateObject {
-        public List<RunAndMaxFrameSizePair> runAndMaxFrameSizePairs;
+        public List<GeneratedRunFileReader> generatedRunFileReaders;
         public ISorter sorter;
 
         public SortTaskState(JobId jobId, TaskId taskId) {
@@ -142,7 +143,7 @@
                     SortTaskState state = new SortTaskState(ctx.getJobletContext().getJobId(),
                             new TaskId(getActivityId(), partition));
                     runGen.close();
-                    state.runAndMaxFrameSizePairs = runGen.getRuns();
+                    state.generatedRunFileReaders = runGen.getRuns();
                     state.sorter = runGen.getSorter();
                     if (LOGGER.isLoggable(Level.INFO)) {
                         LOGGER.info("InitialNumberOfRuns:" + runGen.getRuns().size());
@@ -166,9 +167,9 @@
             super(id);
         }
 
-        protected abstract ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
+        protected abstract AbstractExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
                 IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter,
-                List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+                List<GeneratedRunFileReader> runs, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
                 int necessaryFrames);
 
         @Override
@@ -180,17 +181,16 @@
                 public void initialize() throws HyracksDataException {
                     SortTaskState state = (SortTaskState) ctx
                             .getStateObject(new TaskId(new ActivityId(getOperatorId(), SORT_ACTIVITY_ID), partition));
-                    List<RunAndMaxFrameSizePair> runs = state.runAndMaxFrameSizePairs;
+                    List<GeneratedRunFileReader> runs = state.generatedRunFileReaders;
                     ISorter sorter = state.sorter;
                     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();
-                    ExternalSortRunMerger merger = getSortRunMerger(ctx, recordDescProvider, writer, sorter, runs,
-                            comparators, nmkComputer, framesLimit);
+                    INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
+                            : firstKeyNormalizerFactory.createNormalizedKeyComputer();
+                    AbstractExternalSortRunMerger merger = getSortRunMerger(ctx, recordDescProvider, writer, sorter,
+                            runs, comparators, nmkComputer, framesLimit);
                     merger.process();
                 }
             };
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index fd9187d..36d0c1b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -18,15 +18,21 @@
  */
 package org.apache.hyracks.dataflow.std.sort;
 
+import java.util.List;
+
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.ActivityId;
-import org.apache.hyracks.api.dataflow.value.*;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
-
-import java.util.List;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
+import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
 
 public class ExternalSortOperatorDescriptor extends AbstractSorterOperatorDescriptor {
 
@@ -61,8 +67,8 @@
             @Override
             protected AbstractSortRunGenerator getRunGenerator(IHyracksTaskContext ctx,
                     IRecordDescriptorProvider recordDescProvider) throws HyracksDataException {
-                return new ExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory,
-                        comparatorFactories, recordDescriptors[0], alg, policy, framesLimit, outputLimit);
+                return new ExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
+                        recordDescriptors[0], alg, policy, framesLimit, outputLimit);
             }
         };
     }
@@ -72,10 +78,11 @@
         return new AbstractSorterOperatorDescriptor.MergeActivity(id) {
             @Override
             protected ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
-                    IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter, List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators,
+                    IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter,
+                    List<GeneratedRunFileReader> runs, IBinaryComparator[] comparators,
                     INormalizedKeyComputer nmkComputer, int necessaryFrames) {
-                return new ExternalSortRunMerger(ctx, sorter, runs, sortFields, comparators,
-                        nmkComputer, recordDescriptors[0], necessaryFrames, outputLimit, writer);
+                return new ExternalSortRunMerger(ctx, sorter, runs, sortFields, comparators, nmkComputer,
+                        recordDescriptors[0], necessaryFrames, outputLimit, writer);
             }
         };
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index a8ab898..3075719 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -18,8 +18,6 @@
  */
 package org.apache.hyracks.dataflow.std.sort;
 
-import java.nio.ByteBuffer;
-
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -28,20 +26,9 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotBiggestFirst;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotLastFit;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotSmallestFit;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.IFrameFreeSlotPolicy;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
 
-public class ExternalSortRunGenerator extends AbstractSortRunGenerator {
-
-    protected final IHyracksTaskContext ctx;
-    protected final IFrameSorter frameSorter;
-    protected final int maxSortFrames;
+public class ExternalSortRunGenerator extends AbstractExternalSortRunGenerator {
 
     public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
@@ -62,40 +49,8 @@
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDesc, Algorithm alg, EnumFreeSlotPolicy policy, int framesLimit, int outputLimit)
                     throws HyracksDataException {
-        this.ctx = ctx;
-        maxSortFrames = framesLimit - 1;
-
-        IFrameFreeSlotPolicy freeSlotPolicy = null;
-        switch (policy) {
-            case SMALLEST_FIT:
-                freeSlotPolicy = new FrameFreeSlotSmallestFit();
-                break;
-            case LAST_FIT:
-                freeSlotPolicy = new FrameFreeSlotLastFit(maxSortFrames);
-                break;
-            case BIGGEST_FIT:
-                freeSlotPolicy = new FrameFreeSlotBiggestFirst(maxSortFrames);
-                break;
-        }
-        IFrameBufferManager bufferManager = new VariableFrameMemoryManager(
-                new VariableFramePool(ctx, maxSortFrames * ctx.getInitialFrameSize()), freeSlotPolicy);
-        if (alg == Algorithm.MERGE_SORT) {
-            frameSorter = new FrameSorterMergeSort(ctx, bufferManager, sortFields, firstKeyNormalizerFactory,
-                    comparatorFactories, recordDesc, outputLimit);
-        } else {
-            frameSorter = new FrameSorterQuickSort(ctx, bufferManager, sortFields, firstKeyNormalizerFactory,
-                    comparatorFactories, recordDesc, outputLimit);
-        }
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        if (!frameSorter.insertFrame(buffer)) {
-            flushFramesToRun();
-            if (!frameSorter.insertFrame(buffer)) {
-                throw new HyracksDataException("The given frame is too big to insert into the sorting memory.");
-            }
-        }
+        super(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc, alg, policy, framesLimit,
+                outputLimit);
     }
 
     @Override
@@ -110,8 +65,4 @@
         return writer;
     }
 
-    @Override
-    public ISorter getSorter() {
-        return frameSorter;
-    }
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index d44826e..e773a3b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -18,255 +18,54 @@
  */
 package org.apache.hyracks.dataflow.std.sort;
 
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.LinkedList;
 import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
-import org.apache.hyracks.api.comm.FrameConstants;
-import org.apache.hyracks.api.comm.IFrameReader;
 import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.VSizeFrame;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
-import org.apache.hyracks.dataflow.std.sort.util.GroupVSizeFrame;
 
-public class ExternalSortRunMerger {
+public class ExternalSortRunMerger extends AbstractExternalSortRunMerger {
 
-    protected final IHyracksTaskContext ctx;
-    protected final IFrameWriter writer;
-
-    private final List<RunAndMaxFrameSizePair> runs;
-    private final BitSet currentGenerationRunAvailable;
     private final int[] sortFields;
-    private final IBinaryComparator[] comparators;
-    private final INormalizedKeyComputer nmkComputer;
-    private final RecordDescriptor recordDesc;
-    private final int framesLimit;
-    private final int MAX_FRAME_SIZE;
-    private final ArrayList<IFrameReader> tempRuns;
-    private final int topK;
-    private List<GroupVSizeFrame> inFrames;
-    private VSizeFrame outputFrame;
-    private ISorter sorter;
 
-    private static final Logger LOGGER = Logger.getLogger(ExternalSortRunMerger.class.getName());
-
-    public ExternalSortRunMerger(IHyracksTaskContext ctx, ISorter sorter, List<RunAndMaxFrameSizePair> runs,
-            int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
-            RecordDescriptor recordDesc, int framesLimit, IFrameWriter writer) {
-        this(ctx, sorter, runs, sortFields, comparators, nmkComputer, recordDesc, framesLimit, Integer.MAX_VALUE,
-                writer);
-    }
-
-    public ExternalSortRunMerger(IHyracksTaskContext ctx, ISorter sorter, List<RunAndMaxFrameSizePair> runs,
+    public ExternalSortRunMerger(IHyracksTaskContext ctx, ISorter sorter, List<GeneratedRunFileReader> runs,
             int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
             RecordDescriptor recordDesc, int framesLimit, int topK, IFrameWriter writer) {
-        this.ctx = ctx;
-        this.sorter = sorter;
-        this.runs = new LinkedList<>(runs);
-        this.currentGenerationRunAvailable = new BitSet(runs.size());
+        super(ctx, sorter, runs, comparators, nmkComputer, recordDesc, framesLimit, topK, writer);
         this.sortFields = sortFields;
-        this.comparators = comparators;
-        this.nmkComputer = nmkComputer;
-        this.recordDesc = recordDesc;
-        this.framesLimit = framesLimit;
-        this.writer = writer;
-        this.MAX_FRAME_SIZE = FrameConstants.MAX_NUM_MINFRAME * ctx.getInitialFrameSize();
-        this.topK = topK;
-        this.tempRuns = new ArrayList<>(runs.size());
     }
 
-    public void process() throws HyracksDataException {
-        IFrameWriter finalWriter = null;
-        try {
-            if (runs.size() <= 0) {
-                finalWriter = prepareSkipMergingFinalResultWriter(writer);
-                finalWriter.open();
-                if (sorter != null) {
-                    if (sorter.hasRemaining()) {
-                        sorter.flush(finalWriter);
-                    }
-                    sorter.close();
-                }
-            } else {
-                /** recycle sort buffer */
-                if (sorter != null) {
-                    sorter.close();
-                }
-
-                finalWriter = prepareFinalMergeResultWriter(writer);
-                finalWriter.open();
-
-                int maxMergeWidth = framesLimit - 1;
-
-                inFrames = new ArrayList<>(maxMergeWidth);
-                outputFrame = new VSizeFrame(ctx);
-                List<RunAndMaxFrameSizePair> partialRuns = new ArrayList<>(maxMergeWidth);
-
-                int stop = runs.size();
-                currentGenerationRunAvailable.set(0, stop);
-
-                while (true) {
-
-                    int unUsed = selectPartialRuns(maxMergeWidth * ctx.getInitialFrameSize(), runs, partialRuns,
-                            currentGenerationRunAvailable, stop);
-                    prepareFrames(unUsed, inFrames, partialRuns);
-
-                    if (!currentGenerationRunAvailable.isEmpty() || stop < runs.size()) {
-                        IFrameReader reader;
-                        int mergedMaxFrameSize;
-                        if (partialRuns.size() == 1) {
-                            if (!currentGenerationRunAvailable.isEmpty()) {
-                                throw new HyracksDataException(
-                                        "The record is too big to put into the merging frame, please"
-                                                + " allocate more sorting memory");
-                            } else {
-                                reader = partialRuns.get(0).run;
-                                mergedMaxFrameSize = partialRuns.get(0).maxFrameSize;
-                            }
-
-                        } else {
-                            RunFileWriter mergeFileWriter = prepareIntermediateMergeRunFile();
-                            IFrameWriter mergeResultWriter = prepareIntermediateMergeResultWriter(mergeFileWriter);
-
-                            mergeResultWriter.open();
-                            mergedMaxFrameSize = merge(mergeResultWriter, partialRuns);
-                            mergeResultWriter.close();
-
-                            reader = mergeFileWriter.createDeleteOnCloseReader();
-                        }
-
-                        appendNewRuns(reader, mergedMaxFrameSize);
-                        if (currentGenerationRunAvailable.isEmpty()) {
-
-                            if (LOGGER.isLoggable(Level.FINE)) {
-                                LOGGER.fine("generated runs:" + stop);
-                            }
-                            runs.subList(0, stop).clear();
-                            currentGenerationRunAvailable.clear();
-                            currentGenerationRunAvailable.set(0, runs.size());
-                            stop = runs.size();
-                        }
-                    } else {
-                        if (LOGGER.isLoggable(Level.FINE)) {
-                            LOGGER.fine("final runs:" + stop);
-                        }
-                        merge(finalWriter, partialRuns);
-                        break;
-                    }
-                }
-            }
-        } catch (Exception e) {
-            finalWriter.fail();
-            throw new HyracksDataException(e);
-        } finally {
-            finalWriter.close();
-        }
-    }
-
-    private void appendNewRuns(IFrameReader reader, int mergedPartialMaxSize) {
-        runs.add(new RunAndMaxFrameSizePair(reader, mergedPartialMaxSize));
-    }
-
-    private static int selectPartialRuns(int budget, List<RunAndMaxFrameSizePair> runs,
-            List<RunAndMaxFrameSizePair> partialRuns, BitSet runAvailable, int stop) {
-        partialRuns.clear();
-        int maxFrameSizeOfGenRun = 0;
-        int nextRunId = runAvailable.nextSetBit(0);
-        while (budget > 0 && nextRunId >= 0 && nextRunId < stop) {
-            int runFrameSize = runs.get(nextRunId).maxFrameSize;
-            if (budget - runFrameSize >= 0) {
-                partialRuns.add(runs.get(nextRunId));
-                budget -= runFrameSize;
-                runAvailable.clear(nextRunId);
-                maxFrameSizeOfGenRun = runFrameSize > maxFrameSizeOfGenRun ? runFrameSize : maxFrameSizeOfGenRun;
-            }
-            nextRunId = runAvailable.nextSetBit(nextRunId + 1);
-        }
-        return budget;
-    }
-
-    private void prepareFrames(int extraFreeMem, List<GroupVSizeFrame> inFrames, List<RunAndMaxFrameSizePair> patialRuns)
-            throws HyracksDataException {
-        if (extraFreeMem > 0 && patialRuns.size() > 1) {
-            int extraFrames = extraFreeMem / ctx.getInitialFrameSize();
-            int avg = (extraFrames / patialRuns.size()) * ctx.getInitialFrameSize();
-            int residue = (extraFrames % patialRuns.size());
-            for (int i = 0; i < residue; i++) {
-                patialRuns.get(i).updateSize(
-                        Math.min(MAX_FRAME_SIZE, patialRuns.get(i).maxFrameSize + avg + ctx.getInitialFrameSize()));
-            }
-            for (int i = residue; i < patialRuns.size() && avg > 0; i++) {
-                patialRuns.get(i).updateSize(Math.min(MAX_FRAME_SIZE, patialRuns.get(i).maxFrameSize + avg));
-            }
-        }
-
-        if (inFrames.size() > patialRuns.size()) {
-            inFrames.subList(patialRuns.size(), inFrames.size()).clear();
-        }
-        int i;
-        for (i = 0; i < inFrames.size(); i++) {
-            inFrames.get(i).resize(patialRuns.get(i).maxFrameSize);
-        }
-        for (; i < patialRuns.size(); i++) {
-            inFrames.add(new GroupVSizeFrame(ctx, patialRuns.get(i).maxFrameSize));
-        }
-    }
-
+    @Override
     protected IFrameWriter prepareSkipMergingFinalResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
         return nextWriter;
     }
 
+    @Override
     protected RunFileWriter prepareIntermediateMergeRunFile() throws HyracksDataException {
         FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
         return new RunFileWriter(newRun, ctx.getIOManager());
     }
 
+    @Override
     protected IFrameWriter prepareIntermediateMergeResultWriter(RunFileWriter mergeFileWriter)
             throws HyracksDataException {
         return mergeFileWriter;
     }
 
+    @Override
     protected IFrameWriter prepareFinalMergeResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
         return nextWriter;
     }
 
+    @Override
     protected int[] getSortFields() {
         return sortFields;
     }
 
-    private int merge(IFrameWriter writer, List<RunAndMaxFrameSizePair> partialRuns) throws HyracksDataException {
-        tempRuns.clear();
-        for (int i = 0; i < partialRuns.size(); i++) {
-            tempRuns.add(partialRuns.get(i).run);
-        }
-        RunMergingFrameReader merger = new RunMergingFrameReader(ctx, tempRuns, inFrames, getSortFields(), comparators,
-                nmkComputer, recordDesc, topK);
-        int maxFrameSize = 0;
-        int io = 0;
-        merger.open();
-        try {
-            while (merger.nextFrame(outputFrame)) {
-                FrameUtils.flushFrame(outputFrame.getBuffer(), writer);
-                maxFrameSize = maxFrameSize < outputFrame.getFrameSize() ? outputFrame.getFrameSize() : maxFrameSize;
-                io++;
-            }
-        } finally {
-            merger.close();
-            if (LOGGER.isLoggable(Level.FINE)) {
-                LOGGER.fine("Output " + io + " frames");
-            }
-        }
-        return maxFrameSize;
-    }
-
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
index 5ed11b9..f1f1b1b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
@@ -18,21 +18,16 @@
  */
 package org.apache.hyracks.dataflow.std.sort;
 
-import java.nio.ByteBuffer;
-
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.util.IntSerDeUtils;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.IFrameBufferManager;
 
 public class FrameSorterMergeSort extends AbstractFrameSorter {
 
     private int[] tPointersTemp;
-    private FrameTupleAccessor fta2;
 
     public FrameSorterMergeSort(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
@@ -46,7 +41,6 @@
             RecordDescriptor recordDescriptor, int outputLimit) throws HyracksDataException {
         super(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
                 outputLimit);
-        fta2 = new FrameTupleAccessor(recordDescriptor);
     }
 
     @Override
@@ -122,42 +116,4 @@
         tPointersTemp[dest * 4 + 3] = tPointers[src * 4 + 3];
     }
 
-    private int compare(int tp1, int tp2) throws HyracksDataException {
-        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;
-        }
-        int i2 = tp2i;
-        int j2 = tp2j;
-        ByteBuffer buf1 = super.bufferManager.getFrame(i1);
-        ByteBuffer buf2 = super.bufferManager.getFrame(i2);
-        byte[] b1 = buf1.array();
-        byte[] b2 = buf2.array();
-        inputTupleAccessor.reset(buf1);
-        fta2.reset(buf2);
-        for (int f = 0; f < comparators.length; ++f) {
-            int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(b1, j1 + (fIdx - 1) * 4);
-            int f1End = IntSerDeUtils.getInt(b1, j1 + fIdx * 4);
-            int s1 = j1 + inputTupleAccessor.getFieldSlotsLength() + f1Start;
-            int l1 = f1End - f1Start;
-            int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(b2, j2 + (fIdx - 1) * 4);
-            int f2End = IntSerDeUtils.getInt(b2, j2 + fIdx * 4);
-            int s2 = j2 + fta2.getFieldSlotsLength() + f2Start;
-            int l2 = f2End - f2Start;
-            int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
-            if (c != 0) {
-                return c;
-            }
-        }
-        return 0;
-    }
-
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterQuickSort.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
index a970a0e..cf864f6 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
@@ -18,20 +18,15 @@
  */
 package org.apache.hyracks.dataflow.std.sort;
 
-import java.nio.ByteBuffer;
-
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.IFrameBufferManager;
 
 public class FrameSorterQuickSort extends AbstractFrameSorter {
 
-    private FrameTupleAccessor fta2;
-
     public FrameSorterQuickSort(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor) throws HyracksDataException {
@@ -44,19 +39,15 @@
             RecordDescriptor recordDescriptor, int outputLimit) throws HyracksDataException {
         super(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
                 outputLimit);
-        fta2 = new FrameTupleAccessor(recordDescriptor);
     }
 
     @Override
     void sortTupleReferences() throws HyracksDataException {
-        sort(tPointers, 0, tupleCount);
+        sort(0, tupleCount);
     }
 
-    void sort(int[] tPointers, int offset, int length) throws HyracksDataException {
+    void sort(int offset, int length) throws HyracksDataException {
         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;
@@ -64,7 +55,7 @@
         int d = c;
         while (true) {
             while (b <= c) {
-                int cmp = compare(tPointers, b, mi, mj, mv);
+                int cmp = compare(b, m);
                 if (cmp > 0) {
                     break;
                 }
@@ -74,7 +65,7 @@
                 ++b;
             }
             while (c >= b) {
-                int cmp = compare(tPointers, c, mi, mj, mv);
+                int cmp = compare(c, m);
                 if (cmp < 0) {
                     break;
                 }
@@ -96,10 +87,10 @@
         vecswap(tPointers, b, n - s, s);
 
         if ((s = b - a) > 1) {
-            sort(tPointers, offset, s);
+            sort(offset, s);
         }
         if ((s = d - c) > 1) {
-            sort(tPointers, n - s, s);
+            sort(n - s, s);
         }
     }
 
@@ -117,37 +108,4 @@
         }
     }
 
-    private int compare(int[] tPointers, int tp1, int tp2i, int tp2j, int tp2v) throws HyracksDataException {
-        int i1 = tPointers[tp1 * 4];
-        int j1 = tPointers[tp1 * 4 + 1];
-        int v1 = tPointers[tp1 * 4 + 3];
-        if (v1 != tp2v) {
-            return ((((long) v1) & 0xffffffffL) < (((long) tp2v) & 0xffffffffL)) ? -1 : 1;
-        }
-        int i2 = tp2i;
-        int j2 = tp2j;
-        ByteBuffer buf1 = super.bufferManager.getFrame(i1);
-        ByteBuffer buf2 = super.bufferManager.getFrame(i2);
-        byte[] b1 = buf1.array();
-        byte[] b2 = buf2.array();
-        inputTupleAccessor.reset(buf1);
-        fta2.reset(buf2);
-        for (int f = 0; f < comparators.length; ++f) {
-            int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : buf1.getInt(j1 + (fIdx - 1) * 4);
-            int f1End = buf1.getInt(j1 + fIdx * 4);
-            int s1 = j1 + inputTupleAccessor.getFieldSlotsLength() + f1Start;
-            int l1 = f1End - f1Start;
-            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);
-            if (c != 0) {
-                return c;
-            }
-        }
-        return 0;
-    }
-
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HeapSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HeapSortRunGenerator.java
index 7f2847c..648be02 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HeapSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HeapSortRunGenerator.java
@@ -31,10 +31,10 @@
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.IFramePool;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.ITupleBufferManager;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableTupleMemoryManager;
+import org.apache.hyracks.dataflow.std.buffermanager.IFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.IDeletableTupleBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableDeletableTupleMemoryManager;
 
 public class HeapSortRunGenerator extends AbstractSortRunGenerator {
     protected final IHyracksTaskContext ctx;
@@ -64,7 +64,7 @@
     @Override
     public void open() throws HyracksDataException {
         IFramePool framePool = new VariableFramePool(ctx, (frameLimit - 1) * ctx.getInitialFrameSize());
-        ITupleBufferManager bufferManager = new VariableTupleMemoryManager(framePool, recordDescriptor);
+        IDeletableTupleBufferManager bufferManager = new VariableDeletableTupleMemoryManager(framePool, recordDescriptor);
         tupleSorter = new TupleSorterHeapSort(ctx, bufferManager, topK, sortFields, nmkFactory, comparatorFactories);
         super.open();
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
index af271ff..9be4bc6 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
@@ -30,9 +30,10 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.dataflow.common.io.RunFileWriter;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotBiggestFirst;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
+import org.apache.hyracks.dataflow.std.buffermanager.FrameFreeSlotPolicyFactory;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFrameMemoryManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFramePool;
 
 public class HybridTopKSortRunGenerator extends HeapSortRunGenerator {
     private static final Logger LOG = Logger.getLogger(HybridTopKSortRunGenerator.class.getName());
@@ -59,8 +60,8 @@
 
     @Override
     protected RunFileWriter getRunFileWriter() throws HyracksDataException {
-        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
-                HybridTopKSortRunGenerator.class.getSimpleName());
+        FileReference file = ctx.getJobletContext()
+                .createManagedWorkspaceFile(HybridTopKSortRunGenerator.class.getSimpleName());
         return new RunFileWriter(file, ctx.getIOManager());
     }
 
@@ -95,7 +96,8 @@
             if (frameSorter == null) {
                 VariableFrameMemoryManager bufferManager = new VariableFrameMemoryManager(
                         new VariableFramePool(ctx, (frameLimit - 1) * ctx.getInitialFrameSize()),
-                        new FrameFreeSlotBiggestFirst(frameLimit - 1));
+                        FrameFreeSlotPolicyFactory.createFreeSlotPolicy(EnumFreeSlotPolicy.BIGGEST_FIT,
+                                frameLimit - 1));
                 frameSorter = new FrameSorterMergeSort(ctx, bufferManager, sortFields, nmkFactory, comparatorFactories,
                         recordDescriptor, topK);
                 if (LOG.isLoggable(Level.FINE)) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/IRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/IRunGenerator.java
index 70075b1..02880dd 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/IRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/IRunGenerator.java
@@ -21,6 +21,7 @@
 import java.util.List;
 
 import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
 
 /**
  * @author pouria
@@ -31,5 +32,5 @@
     /**
      * @return the list of generated (sorted) runs
      */
-    List<RunAndMaxFrameSizePair> getRuns();
+    List<GeneratedRunFileReader> getRuns();
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ISorter.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ISorter.java
index 5f83a79..3fb2b54 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ISorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ISorter.java
@@ -30,7 +30,7 @@
 
     void sort() throws HyracksDataException;
 
-    void close();
+    void close() throws HyracksDataException;
 
     int flush(IFrameWriter writer) throws HyracksDataException;
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index 2ffe437..521c2ae 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -40,10 +40,11 @@
 import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
 import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotLastFit;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
+import org.apache.hyracks.dataflow.std.buffermanager.FrameFreeSlotPolicyFactory;
+import org.apache.hyracks.dataflow.std.buffermanager.IFrameBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFrameMemoryManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableFramePool;
 
 public class InMemorySortOperatorDescriptor extends AbstractOperatorDescriptor {
     private static final long serialVersionUID = 1L;
@@ -119,7 +120,8 @@
                             new TaskId(getActivityId(), partition));
 
                     IFrameBufferManager frameBufferManager = new VariableFrameMemoryManager(
-                            new VariableFramePool(ctx, VariableFramePool.UNLIMITED_MEMORY), new FrameFreeSlotLastFit());
+                            new VariableFramePool(ctx, VariableFramePool.UNLIMITED_MEMORY),
+                            FrameFreeSlotPolicyFactory.createFreeSlotPolicy(EnumFreeSlotPolicy.LAST_FIT));
 
                     state.frameSorter = new FrameSorterMergeSort(ctx, frameBufferManager, sortFields,
                             firstKeyNormalizerFactory, comparatorFactories, recordDescriptors[0]);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
index cd0008f..5df09c6 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
@@ -19,13 +19,19 @@
 
 package org.apache.hyracks.dataflow.std.sort;
 
+import java.util.List;
+
 import org.apache.hyracks.api.comm.IFrameWriter;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.ActivityId;
-import org.apache.hyracks.api.dataflow.value.*;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
-
-import java.util.List;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
 
 public class TopKSorterOperatorDescriptor extends AbstractSorterOperatorDescriptor {
 
@@ -56,10 +62,11 @@
         return new MergeActivity(id) {
             @Override
             protected ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
-                    IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter, List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators,
+                    IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter,
+                    List<GeneratedRunFileReader> runs, IBinaryComparator[] comparators,
                     INormalizedKeyComputer nmkComputer, int necessaryFrames) {
-                return new ExternalSortRunMerger(ctx, sorter, runs, sortFields, comparators,
-                        nmkComputer, recordDescriptors[0], necessaryFrames, topK, writer);
+                return new ExternalSortRunMerger(ctx, sorter, runs, sortFields, comparators, nmkComputer,
+                        recordDescriptors[0], necessaryFrames, topK, writer);
             }
         };
     }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
index fc15bbc..a5ec88a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
@@ -36,8 +36,8 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.ITupleBufferAccessor;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.ITupleBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.IDeletableTupleBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.ITuplePointerAccessor;
 import org.apache.hyracks.dataflow.std.structures.IResetableComparable;
 import org.apache.hyracks.dataflow.std.structures.IResetableComparableFactory;
 import org.apache.hyracks.dataflow.std.structures.MaxHeap;
@@ -70,8 +70,8 @@
             }
             bufferAccessor1.reset(tuplePointer);
             bufferAccessor2.reset(o.tuplePointer);
-            byte[] b1 = bufferAccessor1.getTupleBuffer().array();
-            byte[] b2 = bufferAccessor2.getTupleBuffer().array();
+            byte[] b1 = bufferAccessor1.getBuffer().array();
+            byte[] b2 = bufferAccessor2.getBuffer().array();
 
             for (int f = 0; f < comparators.length; ++f) {
                 int fIdx = sortFields[f];
@@ -104,9 +104,9 @@
         }
     }
 
-    private final ITupleBufferManager bufferManager;
-    private final ITupleBufferAccessor bufferAccessor1;
-    private final ITupleBufferAccessor bufferAccessor2;
+    private final IDeletableTupleBufferManager bufferManager;
+    private final ITuplePointerAccessor bufferAccessor1;
+    private final ITuplePointerAccessor bufferAccessor2;
     private final int topK;
     private final FrameTupleAppender outputAppender;
     private final IFrame outputFrame;
@@ -120,12 +120,13 @@
     private MaxHeap heap;
     private boolean isSorted;
 
-    public TupleSorterHeapSort(IHyracksTaskContext ctx, ITupleBufferManager bufferManager, int topK, int[] sortFields,
+    public TupleSorterHeapSort(IHyracksTaskContext ctx, IDeletableTupleBufferManager bufferManager, int topK,
+            int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories)
             throws HyracksDataException {
         this.bufferManager = bufferManager;
-        this.bufferAccessor1 = bufferManager.getTupleAccessor();
-        this.bufferAccessor2 = bufferManager.getTupleAccessor();
+        this.bufferAccessor1 = bufferManager.createTupleAccessor();
+        this.bufferAccessor2 = bufferManager.createTupleAccessor();
         this.topK = topK;
         this.outputFrame = new VSizeFrame(ctx);
         this.outputAppender = new FrameTupleAppender();
@@ -190,7 +191,7 @@
         }
         bufferAccessor2.reset(maxEntry.tuplePointer);
         byte[] b1 = frameTupleAccessor.getBuffer().array();
-        byte[] b2 = bufferAccessor2.getTupleBuffer().array();
+        byte[] b2 = bufferAccessor2.getBuffer().array();
 
         for (int f = 0; f < comparators.length; ++f) {
             int fIdx = sortFields[f];
@@ -236,7 +237,7 @@
     };
 
     @Override
-    public void close() {
+    public void close() throws HyracksDataException {
         heap = null;
         bufferManager.close();
         isSorted = false;
@@ -254,7 +255,7 @@
             HeapEntry minEntry = (HeapEntry) entries[i];
             bufferAccessor1.reset(minEntry.tuplePointer);
             int flushed = FrameUtils
-                    .appendToWriter(writer, outputAppender, bufferAccessor1.getTupleBuffer().array(),
+                    .appendToWriter(writer, outputAppender, bufferAccessor1.getBuffer().array(),
                             bufferAccessor1.getTupleStartOffset(), bufferAccessor1.getTupleLength());
             if (flushed > 0) {
                 maxFrameSize = Math.max(maxFrameSize, flushed);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
index f77ec63..31ea07d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
@@ -26,7 +26,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Basically it a union of the {@link IFrameTupleAccessor} and {@link IFrameTupleAppender}.
+ * Basically, it is an union of the {@link IFrameTupleAccessor} and {@link IFrameTupleAppender}.
  * Moreover, it has the delete function as well.
  * This is a special TupleAccessor used for TopK sorting.
  * In HeapSort, or other Tuple-based operators, we need to append the tuple, access the arbitrary previously
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/AbstractHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/AbstractHeap.java
index c6ca09b..d9a62c8 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/AbstractHeap.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/AbstractHeap.java
@@ -72,9 +72,7 @@
 
     @Override
     public void reset() {
-        for (int i = 0; i < numEntry; i++) {
-            entries[i] = null;
-        }
+        Arrays.fill(entries, null);
         numEntry = 0;
     }
 
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/ISerializableTable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/ISerializableTable.java
index ceae0f1..8cd6792 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/ISerializableTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/ISerializableTable.java
@@ -22,15 +22,19 @@
 
 public interface ISerializableTable {
 
-    public void insert(int entry, TuplePointer tuplePointer) throws HyracksDataException;
+    void insert(int entry, TuplePointer tuplePointer) throws HyracksDataException;
 
-    public void getTuplePointer(int entry, int offset, TuplePointer tuplePointer);
+    void delete(int entry);
 
-    public int getFrameCount();
+    boolean getTuplePointer(int entry, int offset, TuplePointer tuplePointer);
 
-    public int getTupleCount();
+    int getFrameCount();
 
-    public void reset();
+    int getTupleCount();
 
-    public void close();
+    int getTupleCount(int entry);
+
+    void reset();
+
+    void close();
 }
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTable.java
index 7db57c0..1f2ebef 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTable.java
@@ -21,6 +21,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.hyracks.api.context.IHyracksFrameMgrContext;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
@@ -36,14 +37,14 @@
     private IntSerDeBuffer[] headers;
     private List<IntSerDeBuffer> contents = new ArrayList<IntSerDeBuffer>();
     private List<Integer> frameCurrentIndex = new ArrayList<Integer>();
-    private final IHyracksTaskContext ctx;
-    private int frameCapacity = 0;
+    private final IHyracksFrameMgrContext ctx;
+    private final int frameCapacity;
     private int currentLargestFrameIndex = 0;
     private int tupleCount = 0;
     private int headerFrameCount = 0;
     private TuplePointer tempTuplePointer = new TuplePointer();
 
-    public SerializableHashTable(int tableSize, final IHyracksTaskContext ctx) throws HyracksDataException {
+    public SerializableHashTable(int tableSize, final IHyracksFrameMgrContext ctx) throws HyracksDataException {
         this.ctx = ctx;
         int frameSize = ctx.getInitialFrameSize();
 
@@ -81,28 +82,45 @@
     }
 
     @Override
-    public void getTuplePointer(int entry, int offset, TuplePointer dataPointer) {
+    public void delete(int entry) {
+        int hFrameIndex = getHeaderFrameIndex(entry);
+        int headerOffset = getHeaderFrameOffset(entry);
+        IntSerDeBuffer header = headers[hFrameIndex];
+        if (header != null) {
+            int frameIndex = header.getInt(headerOffset);
+            int offsetIndex = header.getInt(headerOffset + 1);
+            if (frameIndex >= 0) {
+                IntSerDeBuffer frame = contents.get(frameIndex);
+                int entryUsedItems = frame.getInt(offsetIndex + 1);
+                frame.writeInt(offsetIndex + 1, 0);
+                tupleCount -= entryUsedItems;
+            }
+        }
+    }
+
+    @Override
+    public boolean getTuplePointer(int entry, int offset, TuplePointer dataPointer) {
         int hFrameIndex = getHeaderFrameIndex(entry);
         int headerOffset = getHeaderFrameOffset(entry);
         IntSerDeBuffer header = headers[hFrameIndex];
         if (header == null) {
             dataPointer.frameIndex = -1;
             dataPointer.tupleIndex = -1;
-            return;
+            return false;
         }
         int frameIndex = header.getInt(headerOffset);
         int offsetIndex = header.getInt(headerOffset + 1);
         if (frameIndex < 0) {
             dataPointer.frameIndex = -1;
             dataPointer.tupleIndex = -1;
-            return;
+            return false;
         }
         IntSerDeBuffer frame = contents.get(frameIndex);
         int entryUsedItems = frame.getInt(offsetIndex + 1);
         if (offset > entryUsedItems - 1) {
             dataPointer.frameIndex = -1;
             dataPointer.tupleIndex = -1;
-            return;
+            return false;
         }
         int startIndex = offsetIndex + 2 + offset * 2;
         while (startIndex >= frameCapacity) {
@@ -112,6 +130,7 @@
         frame = contents.get(frameIndex);
         dataPointer.frameIndex = frame.getInt(startIndex);
         dataPointer.tupleIndex = frame.getInt(startIndex + 1);
+        return true;
     }
 
     @Override
@@ -139,9 +158,26 @@
     }
 
     @Override
+    public int getTupleCount(int entry) {
+        int hFrameIndex = getHeaderFrameIndex(entry);
+        int headerOffset = getHeaderFrameOffset(entry);
+        IntSerDeBuffer header = headers[hFrameIndex];
+        if (header != null) {
+            int frameIndex = header.getInt(headerOffset);
+            int offsetIndex = header.getInt(headerOffset + 1);
+            if (frameIndex >= 0) {
+                IntSerDeBuffer frame = contents.get(frameIndex);
+                int entryUsedItems = frame.getInt(offsetIndex + 1);
+                return entryUsedItems;
+            }
+        }
+        return 0;
+    }
+
+    @Override
     public void close() {
         int nFrames = contents.size();
-    	for (int i = 0; i < headers.length; i++)
+        for (int i = 0; i < headers.length; i++)
             headers[i] = null;
         contents.clear();
         frameCurrentIndex.clear();
@@ -259,31 +295,30 @@
         return offset;
     }
 
-}
+    class IntSerDeBuffer {
 
-class IntSerDeBuffer {
+        private byte[] bytes;
 
-    private byte[] bytes;
+        public IntSerDeBuffer(byte[] data) {
+            this.bytes = data;
+        }
 
-    public IntSerDeBuffer(byte[] data) {
-        this.bytes = data;
-    }
+        public int getInt(int pos) {
+            int offset = pos * 4;
+            return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16)
+                    + ((bytes[offset + 2] & 0xff) << 8) + ((bytes[offset + 3] & 0xff) << 0);
+        }
 
-    public int getInt(int pos) {
-        int offset = pos * 4;
-        return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
-                + ((bytes[offset + 3] & 0xff) << 0);
-    }
+        public void writeInt(int pos, int value) {
+            int offset = pos * 4;
+            bytes[offset++] = (byte) (value >> 24);
+            bytes[offset++] = (byte) (value >> 16);
+            bytes[offset++] = (byte) (value >> 8);
+            bytes[offset++] = (byte) (value);
+        }
 
-    public void writeInt(int pos, int value) {
-        int offset = pos * 4;
-        bytes[offset++] = (byte) (value >> 24);
-        bytes[offset++] = (byte) (value >> 16);
-        bytes[offset++] = (byte) (value >> 8);
-        bytes[offset++] = (byte) (value);
-    }
-
-    public int capacity() {
-        return bytes.length / 4;
+        public int capacity() {
+            return bytes.length / 4;
+        }
     }
 }
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/FrameTuplePairComparator.java
similarity index 65%
rename from hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
rename to hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/FrameTuplePairComparator.java
index 602e018..c74fe04 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/FrameTuplePairComparator.java
@@ -16,10 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.dataflow.common.comm.io;
+package org.apache.hyracks.dataflow.std.util;
 
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITuplePointerAccessor;
 
 public class FrameTuplePairComparator {
     private final int[] keys0;
@@ -32,7 +34,7 @@
         this.comparators = comparators;
     }
 
-    public int compare(FrameTupleAccessor accessor0, int tIndex0, FrameTupleAccessor accessor1, int tIndex1)
+    public int compare(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
             throws HyracksDataException {
         int tStart0 = accessor0.getTupleStartOffset(tIndex0);
         int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
@@ -59,4 +61,28 @@
         }
         return 0;
     }
+
+    public int compare(IFrameTupleAccessor accessor0, int tIndex0, ITuplePointerAccessor bufferAccessor)
+            throws HyracksDataException {
+        int tStart0 = accessor0.getTupleStartOffset(tIndex0);
+        int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
+
+        for (int i = 0; i < keys0.length; ++i) {
+            int fIdx0 = keys0[i];
+            int fStart0 = accessor0.getFieldStartOffset(tIndex0, fIdx0);
+            int fEnd0 = accessor0.getFieldEndOffset(tIndex0, fIdx0);
+            int fLen0 = fEnd0 - fStart0;
+
+            int fStart1 = bufferAccessor.getAbsFieldStartOffset(keys1[i]);
+            int fLen1 = bufferAccessor.getFieldLength(keys1[i]);
+
+            int c = comparators[i].compare(accessor0.getBuffer().array(), fStart0 + fStartOffset0, fLen0, bufferAccessor
+                    .getBuffer().array(), fStart1, fLen1);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+
 }
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractFramePoolTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractFramePoolTest.java
new file mode 100644
index 0000000..3e4e578
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractFramePoolTest.java
@@ -0,0 +1,144 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.BUDGET;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.MIN_FRAME_SIZE;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.NUM_MIN_FRAME;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.commonFrameManager;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.junit.Test;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractFramePoolTest {
+    IFramePool pool;
+
+    @Test
+    public void testGetMinFrameSize() throws Exception {
+        assertEquals(MIN_FRAME_SIZE, commonFrameManager.getInitialFrameSize());
+        assertEquals(MIN_FRAME_SIZE, pool.getMinFrameSize());
+    }
+
+    @Test
+    public void testGetMemoryBudgetBytes() throws Exception {
+        assertEquals(BUDGET, pool.getMemoryBudgetBytes());
+    }
+
+    protected void testAllocateShouldFailAfterAllSpaceGetUsed() throws HyracksDataException {
+        for (int i = 0; i < NUM_MIN_FRAME; i++) {
+            assertNull(pool.allocateFrame(MIN_FRAME_SIZE));
+        }
+    }
+
+    protected HashSet<ByteBufferPtr> testAllocateAllSpacesWithMinFrames() throws HyracksDataException {
+        HashSet<ByteBufferPtr> set = new HashSet<>();
+        for (int i = 0; i < NUM_MIN_FRAME; i++) {
+            testAllocateNewBuffer(set, MIN_FRAME_SIZE);
+        }
+        return set;
+    }
+
+    protected void testAllocateNewBuffer(HashSet<ByteBufferPtr> set, int frameSize) throws HyracksDataException {
+        ByteBuffer buffer = pool.allocateFrame(frameSize);
+        assertNotNull(buffer);
+        assertEquals(buffer.capacity(), frameSize);
+        assertTrue(!set.contains(new ByteBufferPtr(buffer)));
+        set.add(new ByteBufferPtr(buffer));
+    }
+
+    /**
+     * Pool will become 1,2,3,4,5
+     *
+     * @throws HyracksDataException
+     */
+    protected Set<ByteBufferPtr> testAllocateVariableFrames() throws HyracksDataException {
+        int budget = BUDGET;
+        int allocate = 0;
+        int i = 1;
+        Set<ByteBufferPtr> set = new HashSet<>();
+        while (budget - allocate >= i * MIN_FRAME_SIZE) {
+            ByteBuffer buffer = pool.allocateFrame(i * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            set.add(new ByteBufferPtr(buffer));
+            allocate += i++ * MIN_FRAME_SIZE;
+        }
+        return set;
+    }
+
+    protected void testShouldFindTheMatchFrames(Set<?> set) throws HyracksDataException {
+        pool.reset();
+        List<Integer> list = Arrays.asList(1, 2, 3, 4, 5);
+
+        for (int i = 0; i < list.size(); i++) {
+            ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(set.contains(new ByteBufferPtr(buffer)));
+            assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
+        }
+        pool.reset();
+        for (int i = list.size() - 1; i >= 0; i--) {
+            ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(set.contains(new ByteBufferPtr(buffer)));
+            assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
+        }
+
+        Collections.shuffle(list);
+        pool.reset();
+        for (int i = 0; i < list.size(); i++) {
+            ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(set.contains(new ByteBufferPtr(buffer)));
+            assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
+        }
+
+    }
+
+    public static class ByteBufferPtr {
+        ByteBuffer bytebuffer;
+
+        public ByteBufferPtr(ByteBuffer buffer) {
+            bytebuffer = buffer;
+        }
+
+        @Override
+        public int hashCode() {
+            return bytebuffer.hashCode();
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            return this.bytebuffer == ((ByteBufferPtr) obj).bytebuffer;
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
new file mode 100644
index 0000000..cdf8834
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import static org.junit.Assert.assertEquals;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.hyracks.api.comm.FixedSizeFrame;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.util.IntSerDeUtils;
+import org.apache.hyracks.dataflow.std.sort.Utility;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public abstract class AbstractTupleMemoryManagerTest {
+    ISerializerDeserializer[] fieldsSerDer = new ISerializerDeserializer[] {
+            IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer() };
+    RecordDescriptor recordDescriptor = new RecordDescriptor(fieldsSerDer);
+    ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
+    FrameTupleAccessor inFTA = new FrameTupleAccessor(recordDescriptor);
+    Random random = new Random(System.currentTimeMillis());
+
+    abstract ITuplePointerAccessor getTupleAccessor();
+
+    protected void assertEachTupleInFTAIsInBuffer(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted) {
+        ITuplePointerAccessor accessor = getTupleAccessor();
+        for (Map.Entry<TuplePointer, Integer> entry : mapInserted.entrySet()) {
+            accessor.reset(entry.getKey());
+            int dataLength = map.get(entry.getValue());
+            assertEquals((int) entry.getValue(),
+                    IntSerDeUtils.getInt(accessor.getBuffer().array(), accessor.getAbsFieldStartOffset(0)));
+            assertEquals(dataLength, accessor.getTupleLength());
+        }
+        assertEquals(map.size(), mapInserted.size());
+    }
+
+    protected Map<Integer, Integer> prepareFixedSizeTuples(
+            int tuplePerFrame,
+            int extraMetaBytePerFrame,
+            int extraMetaBytePerRecord) throws HyracksDataException {
+        Map<Integer, Integer> dataSet = new HashMap<>();
+        ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
+        FixedSizeFrame frame = new FixedSizeFrame(buffer);
+        FrameTupleAppender appender = new FrameTupleAppender();
+        appender.reset(frame, true);
+
+        int sizePerTuple = (Common.MIN_FRAME_SIZE - 1 - tuplePerFrame * 4 - 4 - extraMetaBytePerFrame) / tuplePerFrame;
+        int sizeChar =
+                sizePerTuple - extraMetaBytePerRecord - fieldsSerDer.length * 4 - 4 - 2; //2byte to write str length
+        assert (sizeChar > 0);
+        for (int i = 0; i < Common.NUM_MIN_FRAME * tuplePerFrame; i++) {
+            tupleBuilder.reset();
+            tupleBuilder.addField(fieldsSerDer[0], i);
+            tupleBuilder.addField(fieldsSerDer[1], Utility.repeatString('a', sizeChar));
+            if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+                    tupleBuilder.getSize())) {
+                assert false;
+            }
+            dataSet.put(i, tupleBuilder.getSize() + tupleBuilder.getFieldEndOffsets().length * 4);
+        }
+        inFTA.reset(buffer);
+        return dataSet;
+    }
+
+    protected Map<Integer, Integer> prepareVariableSizeTuples() throws HyracksDataException {
+        Map<Integer, Integer> dataSet = new HashMap<>();
+        ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
+        FixedSizeFrame frame = new FixedSizeFrame(buffer);
+        FrameTupleAppender appender = new FrameTupleAppender();
+        appender.reset(frame, true);
+
+        for (int i = 0; true; i++) {
+            tupleBuilder.reset();
+            tupleBuilder.addField(fieldsSerDer[0], i);
+            tupleBuilder.addField(fieldsSerDer[1], Utility.repeatString('a', i));
+            if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+                    tupleBuilder.getSize())) {
+                break;
+            }
+            dataSet.put(i, tupleBuilder.getSize() + tupleBuilder.getFieldEndOffsets().length * 4);
+        }
+        inFTA.reset(buffer);
+        return dataSet;
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/Common.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/Common.java
similarity index 94%
rename from hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/Common.java
rename to hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/Common.java
index a6f10a6..7389aab 100644
--- a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/Common.java
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/Common.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import org.apache.hyracks.control.nc.resources.memory.FrameManager;
 
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/DeletableFramePoolTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/DeletableFramePoolTest.java
new file mode 100644
index 0000000..2a84e0e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/DeletableFramePoolTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.BUDGET;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.MIN_FRAME_SIZE;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.commonFrameManager;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.util.HashSet;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class DeletableFramePoolTest extends AbstractFramePoolTest {
+
+    @Before
+    public void setUp() {
+        pool = new DeallocatableFramePool(commonFrameManager, BUDGET);
+    }
+
+    DeallocatableFramePool getPool() {
+        return (DeallocatableFramePool) pool;
+    }
+
+    @Test
+    public void testAllocateBuffers() throws HyracksDataException {
+        testAllocateAllSpacesWithMinFrames();
+    }
+
+    @Test
+    public void testCanNotAllocateMore() throws HyracksDataException {
+        testAllocateAllSpacesWithMinFrames();
+        assertNull(pool.allocateFrame(MIN_FRAME_SIZE));
+    }
+
+    @Test
+    public void testReusePreAllocatedBuffer() throws HyracksDataException {
+        HashSet<ByteBufferPtr> set = testAllocateAllSpacesWithMinFrames();
+        for (ByteBufferPtr ptr : set) {
+            getPool().deAllocateBuffer(ptr.bytebuffer);
+        }
+        HashSet<ByteBufferPtr> set2 = testAllocateAllSpacesWithMinFrames();
+        assertEquals(set, set2);
+    }
+
+    @Test
+    public void testMergeCase() throws HyracksDataException {
+        HashSet<ByteBufferPtr> set = testAllocateAllSpacesWithMinFrames();
+        for (ByteBufferPtr ptr : set) {
+            getPool().deAllocateBuffer(ptr.bytebuffer);
+        }
+        set.clear();
+        int i = 1;
+        for (int sum = 0; sum + MIN_FRAME_SIZE * i <= BUDGET; i++) {
+            sum += MIN_FRAME_SIZE * i;
+            testAllocateNewBuffer(set, MIN_FRAME_SIZE * i);
+        }
+        assertNull(pool.allocateFrame(MIN_FRAME_SIZE * i));
+        for (ByteBufferPtr ptr : set) {
+            getPool().deAllocateBuffer(ptr.bytebuffer);
+        }
+        set.clear();
+        testAllocateNewBuffer(set, BUDGET);
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java
similarity index 96%
rename from hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java
rename to hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java
index 58dd7e8..992c7f6 100644
--- a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import static org.junit.Assert.assertEquals;
 
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirstTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirstTest.java
similarity index 96%
rename from hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirstTest.java
rename to hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirstTest.java
index 73eda47..88a54bd 100644
--- a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirstTest.java
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirstTest.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import static junit.framework.Assert.assertEquals;
 
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFitTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFitTest.java
similarity index 97%
rename from hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFitTest.java
rename to hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFitTest.java
index 16338fd..87b1b91 100644
--- a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFitTest.java
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFitTest.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
new file mode 100644
index 0000000..ce31108
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
@@ -0,0 +1,159 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.util.IntSerDeUtils;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+import org.junit.Before;
+import org.junit.Test;
+
+public class VPartitionTupleBufferManagerTest extends AbstractTupleMemoryManagerTest {
+
+    VPartitionTupleBufferManager bufferManager;
+    final int PARTITION = 4;
+
+    @Before
+    public void setup() throws HyracksDataException {
+        IPartitionedMemoryConstrain constrain = new IPartitionedMemoryConstrain() {
+            @Override
+            public int frameLimit(int partitionId) {
+                return Integer.MAX_VALUE;
+            }
+        };
+        bufferManager = new VPartitionTupleBufferManager(Common.commonFrameManager, constrain, PARTITION,
+                Common.BUDGET);
+    }
+
+    @Test
+    public void testGetNumPartitions() throws Exception {
+        assertEquals(PARTITION, bufferManager.getNumPartitions());
+    }
+
+    @Test
+    public void testGetNumTuples() throws Exception {
+        testNumTuplesAndSizeIsZero();
+    }
+
+    @Test
+    public void testInsertToFull() throws HyracksDataException {
+        Map<Integer, Integer> inMap = prepareFixedSizeTuples(10, 0, 0);
+        for (int pid = 0; pid < PARTITION; pid++) {
+            assertInsertOnePartitionToFull(pid, inMap);
+            bufferManager.reset();
+        }
+    }
+
+    @Test
+    public void testInsertClearSequence() throws HyracksDataException {
+        Map<Integer, Integer> inMap = prepareFixedSizeTuples(10, 0, 0);
+        for (int pid = 0; pid < PARTITION; pid++) {
+            assertInsertOnePartitionToFull(pid, inMap);
+            bufferManager.reset();
+        }
+    }
+
+    private void assertInsertOnePartitionToFull(int pid, Map<Integer, Integer> inMap) throws HyracksDataException {
+        testNumTuplesAndSizeIsZero();
+
+        Map<TuplePointer, Integer> outMap = testInsertOnePartitionToFull(pid);
+        assertEquals(outMap.size(), bufferManager.getNumTuples(pid));
+        assertEquals(Common.BUDGET, bufferManager.getPhysicalSize(pid));
+        testCanNotInsertToAnyPartitions();
+        assertEachTupleInFTAIsInBuffer(inMap, outMap);
+
+    }
+
+    private void testCanNotInsertToAnyPartitions() throws HyracksDataException {
+        for (int i = 0; i < PARTITION; i++) {
+            assertFalse(bufferManager.insertTuple(i, tupleBuilder.getByteArray(), tupleBuilder.getFieldEndOffsets(), 0,
+                    tupleBuilder.getSize(), null));
+        }
+    }
+
+    private Map<TuplePointer, Integer> testInsertOnePartitionToFull(int idpart) throws HyracksDataException {
+        Map<TuplePointer, Integer> tuplePointerIntegerMap = new HashMap<>();
+
+        for (int i = 0; i < inFTA.getTupleCount(); i++) {
+            TuplePointer tuplePointer = new TuplePointer();
+            copyDataToTupleBuilder(inFTA, i, tupleBuilder);
+            if (!bufferManager.insertTuple(idpart, tupleBuilder.getByteArray(), tupleBuilder.getFieldEndOffsets(), 0,
+                    tupleBuilder.getSize(), tuplePointer)) {
+                assert false;
+            }
+            tuplePointerIntegerMap.put(tuplePointer,
+                    IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+        }
+        return tuplePointerIntegerMap;
+
+    }
+
+    private static void copyDataToTupleBuilder(FrameTupleAccessor inFTA, int tid, ArrayTupleBuilder tupleBuilder)
+            throws HyracksDataException {
+        tupleBuilder.reset();
+        for (int fid = 0; fid < inFTA.getFieldCount(); fid++) {
+            tupleBuilder.addField(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(tid, fid),
+                    inFTA.getFieldLength(tid, fid));
+        }
+    }
+
+    private void testNumTuplesAndSizeIsZero() {
+        for (int i = 0; i < bufferManager.getNumPartitions(); ++i) {
+            assertEquals(0, bufferManager.getNumTuples(i));
+            assertEquals(0, bufferManager.getPhysicalSize(0));
+        }
+    }
+
+    @Test
+    public void testClearPartition() throws Exception {
+
+        Map<Integer, Integer> inMap = prepareFixedSizeTuples(10, 0, 0);
+        for (int pid = 0; pid < PARTITION; pid++) {
+            assertInsertOnePartitionToFull(pid, inMap);
+            assertClearFullPartitionIsTheSameAsReset();
+        }
+    }
+
+    private void assertClearFullPartitionIsTheSameAsReset() throws HyracksDataException {
+        for (int i = 0; i < PARTITION; i++) {
+            bufferManager.clearPartition(i);
+        }
+        testNumTuplesAndSizeIsZero();
+    }
+
+    @Test
+    public void testClose() throws Exception {
+        testInsertToFull();
+        bufferManager.close();
+    }
+
+    @Override
+    ITuplePointerAccessor getTupleAccessor() {
+        return bufferManager.getTupleAccessor(recordDescriptor);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFramePoolTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFramePoolTest.java
new file mode 100644
index 0000000..8dbe1f0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFramePoolTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.BUDGET;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.MIN_FRAME_SIZE;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.NUM_MIN_FRAME;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.commonFrameManager;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class VariableFramePoolTest extends AbstractFramePoolTest {
+
+    @Before
+    public void setUp() throws Exception {
+        pool = new VariableFramePool(commonFrameManager, BUDGET);
+    }
+
+    @Test
+    public void testAllocateUniformFrameShouldSuccess() throws Exception {
+        testAllocateAllSpacesWithMinFrames();
+        testAllocateShouldFailAfterAllSpaceGetUsed();
+        pool.reset();
+        testAllocateAllSpacesWithMinFrames();
+        pool.close();
+    }
+
+    @Test
+    public void testResetShouldReuseExistingFrames() throws HyracksDataException {
+        Set<?> set1 = testAllocateAllSpacesWithMinFrames();
+        pool.reset();
+        Set<?> set2 = testAllocateAllSpacesWithMinFrames();
+        assertEquals(set1, set2);
+        pool.close();
+    }
+
+    @Test
+    public void testCloseShouldNotReuseExistingFrames() throws HyracksDataException {
+        Set<?> set1 = testAllocateAllSpacesWithMinFrames();
+        pool.close();
+        Set<?> set2 = testAllocateAllSpacesWithMinFrames();
+        assertFalse(set1.equals(set2));
+        pool.close();
+    }
+
+    @Test
+    public void testShouldReturnLargerFramesIfFitOneIsUsed() throws HyracksDataException {
+        Set<?> set = testAllocateVariableFrames();
+        pool.reset();
+        testShouldFindTheMatchFrames(set);
+        pool.reset();
+
+        // allocate seq: 1, 1, 2, 3, 4
+        ByteBuffer placeBuffer = pool.allocateFrame(MIN_FRAME_SIZE);
+        assertTrue(set.contains(new ByteBufferPtr(placeBuffer)));
+        for (int i = 1; i <= 4; i++) {
+            ByteBuffer buffer = pool.allocateFrame(i * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(set.contains(new ByteBufferPtr(buffer)));
+        }
+        assertNull(pool.allocateFrame(MIN_FRAME_SIZE));
+        pool.close();
+    }
+
+    @Test
+    public void testShouldMergeIfNoLargerFrames() throws HyracksDataException {
+        Set<?> set = testAllocateAllSpacesWithMinFrames();
+        pool.reset();
+        int chunks = 5;
+        for (int i = 0; i < NUM_MIN_FRAME; i += chunks) {
+            ByteBuffer buffer = pool.allocateFrame(chunks * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(!set.contains(new ByteBufferPtr(buffer)));
+        }
+    }
+
+    @Test
+    public void testUseMiddleSizeFrameAndNeedToMergeSmallAndBigger() throws HyracksDataException {
+        Set<?> set = testAllocateVariableFrames();
+        pool.reset();
+        // allocate seq: 3, 6, 1;
+        ByteBuffer buffer = pool.allocateFrame(3 * MIN_FRAME_SIZE);
+        assertTrue(set.contains(new ByteBufferPtr(buffer)));
+        buffer = pool.allocateFrame(6 * MIN_FRAME_SIZE);
+        assertFalse(set.contains(new ByteBufferPtr(buffer)));
+        buffer = pool.allocateFrame(1 * MIN_FRAME_SIZE);
+        assertTrue(set.contains(new ByteBufferPtr(buffer)));
+        assertEquals(5 * MIN_FRAME_SIZE, buffer.capacity());
+        pool.reset();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFramesMemoryManagerTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFramesMemoryManagerTest.java
similarity index 93%
rename from hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFramesMemoryManagerTest.java
rename to hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFramesMemoryManagerTest.java
index 18f4f9b..9d4a9a1 100644
--- a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFramesMemoryManagerTest.java
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFramesMemoryManagerTest.java
@@ -17,11 +17,11 @@
  * under the License.
  */
 
-package org.apache.hyracks.dataflow.std.sort.buffermanager;
+package org.apache.hyracks.dataflow.std.buffermanager;
 
-import static org.apache.hyracks.dataflow.std.sort.buffermanager.Common.BUDGET;
-import static org.apache.hyracks.dataflow.std.sort.buffermanager.Common.MIN_FRAME_SIZE;
-import static org.apache.hyracks.dataflow.std.sort.buffermanager.Common.NUM_MIN_FRAME;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.BUDGET;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.MIN_FRAME_SIZE;
+import static org.apache.hyracks.dataflow.std.buffermanager.Common.NUM_MIN_FRAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -66,7 +66,7 @@
     public void testNormalIncomingFrames() throws HyracksDataException {
         HashMap<Integer, Integer> tupleSet = prepareTuples();
         for (IFrame frame : frameList) {
-            assertTrue(framesMemoryManager.insertFrame(frame.getBuffer()) >=0);
+            assertTrue(framesMemoryManager.insertFrame(frame.getBuffer()) >= 0);
         }
         assertEquals(NUM_MIN_FRAME, framesMemoryManager.getNumFrames());
         assertEveryTupleInFTAIsInFrameMemoryManager(tupleSet, framesMemoryManager);
@@ -152,9 +152,10 @@
 
     private void assertEveryTupleInFTAIsInFrameMemoryManager(Map<Integer, Integer> tupleSet,
             VariableFrameMemoryManager framesMemoryManager) {
+        BufferInfo info = new BufferInfo(null, -1, -1);
         for (int i = 0; i < framesMemoryManager.getNumFrames(); ++i) {
-            fta.reset(framesMemoryManager.getFrame(i), framesMemoryManager.getFrameStartOffset(i),
-                    framesMemoryManager.getFrameSize(i));
+            framesMemoryManager.getFrame(i, info);
+            fta.reset(info.getBuffer(), info.getStartOffset(), info.getLength());
             for (int t = 0; t < fta.getTupleCount(); t++) {
                 int id = parseTuple(fta.getBuffer(), fta.getTupleStartOffset(t) + fta.getFieldSlotsLength() + fta
                         .getFieldStartOffset(t, 0));
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
new file mode 100644
index 0000000..036cd41
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
@@ -0,0 +1,181 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.buffermanager;
+
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.hyracks.api.comm.FixedSizeFrame;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.util.IntSerDeUtils;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+import org.junit.Before;
+import org.junit.Test;
+
+public class VariableTupleMemoryManagerTest extends AbstractTupleMemoryManagerTest {
+    VariableDeletableTupleMemoryManager tupleMemoryManager;
+    final int EXTRA_BYTES_FOR_DELETABLE_FRAME = 4;
+
+    @Before
+    public void setup() {
+        VariableFramePool framePool = new VariableFramePool(Common.commonFrameManager, Common.BUDGET);
+        tupleMemoryManager = new VariableDeletableTupleMemoryManager(framePool, recordDescriptor);
+    }
+
+    @Test
+    public void testInsertTupleToMemoryManager() throws HyracksDataException {
+        int iTuplePerFrame = 3;
+        Map<Integer, Integer> mapPrepare = prepareFixedSizeTuples(iTuplePerFrame, EXTRA_BYTES_FOR_DELETABLE_FRAME, 0);
+        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
+        assertEachTupleInFTAIsInBuffer(mapPrepare, mapInserted);
+    }
+
+    @Test
+    public void testReset() throws HyracksDataException {
+        testInsertVariableSizeTupleToMemoryManager();
+        tupleMemoryManager.reset();
+        testInsertTupleToMemoryManager();
+        tupleMemoryManager.reset();
+        testInsertVariableSizeTupleToMemoryManager();
+    }
+
+    @Test
+    public void testDeleteTupleInMemoryManager() throws HyracksDataException {
+        int iTuplePerFrame = 3;
+        Map<Integer, Integer> mapPrepare = prepareFixedSizeTuples(iTuplePerFrame, EXTRA_BYTES_FOR_DELETABLE_FRAME, 0);
+        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
+        deleteRandomSelectedTuples(mapPrepare, mapInserted, 1);
+        assertEachTupleInFTAIsInBuffer(mapPrepare, mapInserted);
+    }
+
+    @Test
+    public void testReOrganizeSpace() throws HyracksDataException {
+        int iTuplePerFrame = 3;
+        Map<Integer, Integer> mapPrepare = prepareFixedSizeTuples(iTuplePerFrame, EXTRA_BYTES_FOR_DELETABLE_FRAME, 0);
+        Map<Integer, Integer> copyMap = new HashMap<>(mapPrepare);
+        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
+        ByteBuffer buffer = deleteRandomSelectedTuples(mapPrepare, mapInserted, mapPrepare.size() / 2);
+        inFTA.reset(buffer);
+        //The deletable frame buffer will keep the deleted slot untouched, which will take more space.
+        // the reason is to not reuse the same TuplePointer outside.
+        Map<TuplePointer, Integer> mapInserted2 = insertInFTAToBufferMayNotAllSuccess();
+        assertTrue(mapInserted2.size() > 0);
+    }
+
+    @Test
+    public void testReOrganizeVariableSizeTuple() throws HyracksDataException {
+        Map<Integer, Integer> map = prepareVariableSizeTuples();
+        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferCouldFailForLargerTuples(map);
+        Map<Integer, Integer> copyMap = new HashMap<>(map);
+
+        ByteBuffer buffer = deleteRandomSelectedTuples(map, mapInserted, map.size() / 2);
+        inFTA.reset(buffer);
+
+        Map<TuplePointer, Integer> mapInserted2 = insertInFTAToBufferCouldFailForLargerTuples(copyMap);
+        Map<TuplePointer, Integer> mergedMap = new HashMap<>(mapInserted);
+        mergedMap.putAll(mapInserted2);
+
+        assertEachTupleInFTAIsInBuffer(copyMap, mergedMap);
+    }
+
+    @Test
+    public void testInsertVariableSizeTupleToMemoryManager() throws HyracksDataException {
+        Map<Integer, Integer> map = prepareVariableSizeTuples();
+        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferCouldFailForLargerTuples(map);
+        assertEachTupleInFTAIsInBuffer(map, mapInserted);
+    }
+
+    @Override
+    ITuplePointerAccessor getTupleAccessor() {
+        return tupleMemoryManager.createTupleAccessor();
+    }
+
+    private Map<TuplePointer, Integer> insertInFTAToBufferShouldAllSuccess() throws HyracksDataException {
+        Map<TuplePointer, Integer> tuplePointerIntegerMap = new HashMap<>();
+        for (int i = 0; i < inFTA.getTupleCount(); i++) {
+            TuplePointer tuplePointer = new TuplePointer();
+            assertTrue(tupleMemoryManager.insertTuple(inFTA, i, tuplePointer));
+            tuplePointerIntegerMap.put(tuplePointer,
+                    IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+        }
+        return tuplePointerIntegerMap;
+    }
+
+    private Map<TuplePointer, Integer> insertInFTAToBufferMayNotAllSuccess() throws HyracksDataException {
+        Map<TuplePointer, Integer> tuplePointerIntegerMap = new HashMap<>();
+        for (int i = 0; i < inFTA.getTupleCount(); i++) {
+            TuplePointer tuplePointer = new TuplePointer();
+            if (!tupleMemoryManager.insertTuple(inFTA, i, tuplePointer)) {
+                break;
+            }
+            tuplePointerIntegerMap.put(tuplePointer,
+                    IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+        }
+        return tuplePointerIntegerMap;
+    }
+
+    private Map<TuplePointer, Integer> insertInFTAToBufferCouldFailForLargerTuples(Map<Integer, Integer> map)
+            throws HyracksDataException {
+        Map<TuplePointer, Integer> tuplePointerIdMap = new HashMap<>();
+        int i = 0;
+        for (; i < inFTA.getTupleCount(); i++) {
+            TuplePointer tuplePointer = new TuplePointer();
+            if (!tupleMemoryManager.insertTuple(inFTA, i, tuplePointer)) {
+                break;
+            }
+            tuplePointerIdMap.put(tuplePointer,
+                    IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+        }
+        for (; i < inFTA.getTupleCount(); i++) {
+            map.remove(IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+        }
+        return tuplePointerIdMap;
+    }
+
+    private ByteBuffer deleteRandomSelectedTuples(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted,
+            int minNumOfRecordTobeDeleted)
+            throws HyracksDataException {
+        ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
+        FixedSizeFrame frame = new FixedSizeFrame(buffer);
+        FrameTupleAppender appender = new FrameTupleAppender();
+        appender.reset(frame, true);
+
+        assert (minNumOfRecordTobeDeleted < mapInserted.size());
+        int countDeleted = minNumOfRecordTobeDeleted + random.nextInt(mapInserted.size() - minNumOfRecordTobeDeleted);
+
+        ITuplePointerAccessor accessor = tupleMemoryManager.createTupleAccessor();
+        for (int i = 0; i < countDeleted; i++) {
+            Iterator<Map.Entry<TuplePointer, Integer>> iter = mapInserted.entrySet().iterator();
+            assert (iter.hasNext());
+            Map.Entry<TuplePointer, Integer> pair = iter.next();
+            accessor.reset(pair.getKey());
+            appender.append(accessor.getBuffer().array(), accessor.getTupleStartOffset(), accessor.getTupleLength());
+            map.remove(pair.getValue());
+            tupleMemoryManager.deleteTuple(pair.getKey());
+            iter.remove();
+        }
+        return buffer;
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFramePoolTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFramePoolTest.java
deleted file mode 100644
index e14c685..0000000
--- a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableFramePoolTest.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * 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 org.apache.hyracks.dataflow.std.sort.buffermanager;
-
-import static org.apache.hyracks.dataflow.std.sort.buffermanager.Common.BUDGET;
-import static org.apache.hyracks.dataflow.std.sort.buffermanager.Common.MIN_FRAME_SIZE;
-import static org.apache.hyracks.dataflow.std.sort.buffermanager.Common.NUM_MIN_FRAME;
-import static org.apache.hyracks.dataflow.std.sort.buffermanager.Common.commonFrameManager;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class VariableFramePoolTest {
-
-    VariableFramePool pool;
-    @Before
-    public void setUp() throws Exception {
-
-        pool = new VariableFramePool(commonFrameManager, BUDGET);
-    }
-
-    @Test
-    public void testGetMinFrameSize() throws Exception {
-        assertEquals(MIN_FRAME_SIZE, commonFrameManager.getInitialFrameSize());
-        assertEquals(MIN_FRAME_SIZE, pool.getMinFrameSize());
-    }
-
-    @Test
-    public void testGetMemoryBudgetBytes() throws Exception {
-        assertEquals(BUDGET, pool.getMemoryBudgetBytes());
-    }
-
-    @Test
-    public void testAllocateUniformFrameShouldSuccess() throws Exception {
-        testAllocateAllSpacesWithMinFrames();
-        testAllocateShouldFailAfterAllSpaceGetUsed();
-        pool.reset();
-        testAllocateAllSpacesWithMinFrames();
-        pool.close();
-    }
-
-    @Test
-    public void testResetShouldReuseExistingFrames() throws HyracksDataException {
-        Set<?> set1 = testAllocateAllSpacesWithMinFrames();
-        pool.reset();
-        Set<?> set2 = testAllocateAllSpacesWithMinFrames();
-        assertEquals(set1, set2);
-        pool.close();
-    }
-
-    @Test
-    public void testCloseShouldNotReuseExistingFrames() throws HyracksDataException {
-        Set<?> set1 = testAllocateAllSpacesWithMinFrames();
-        pool.close();
-        Set<?> set2 = testAllocateAllSpacesWithMinFrames();
-        assertFalse(set1.equals(set2));
-        pool.close();
-    }
-
-    @Test
-    public void testShouldReturnLargerFramesIfFitOneIsUsed() throws HyracksDataException {
-        Set<?> set = testAllocateVariableFrames();
-        pool.reset();
-        testShouldFindTheMatchFrames(set);
-        pool.reset();
-
-        // allocate seq: 1, 1, 2, 3, 4
-        ByteBuffer placeBuffer = pool.allocateFrame(MIN_FRAME_SIZE);
-        assertTrue(set.contains(new ByteBufferPtr(placeBuffer)));
-        for (int i = 1; i <= 4 ; i++) {
-            ByteBuffer buffer = pool.allocateFrame(i * MIN_FRAME_SIZE);
-            assertNotNull(buffer);
-            assertTrue(set.contains(new ByteBufferPtr(buffer)));
-        }
-        assertNull(pool.allocateFrame(MIN_FRAME_SIZE));
-        pool.close();
-    }
-
-    @Test
-    public void testShouldMergeIfNoLargerFrames() throws HyracksDataException {
-        Set<?> set = testAllocateAllSpacesWithMinFrames();
-        pool.reset();
-        int chunks = 5;
-        for (int i = 0; i < NUM_MIN_FRAME; i+= chunks) {
-            ByteBuffer buffer = pool.allocateFrame(chunks * MIN_FRAME_SIZE);
-            assertNotNull(buffer);
-            assertTrue(!set.contains(new ByteBufferPtr(buffer)));
-        }
-    }
-
-    @Test
-    public void testUseMiddleSizeFrameAndNeedToMergeSmallAndBigger() throws HyracksDataException {
-        Set<?> set = testAllocateVariableFrames();
-        pool.reset();
-        // allocate seq: 3, 6, 1;
-        ByteBuffer buffer = pool.allocateFrame(3 * MIN_FRAME_SIZE);
-        assertTrue(set.contains(new ByteBufferPtr(buffer)));
-        buffer = pool.allocateFrame(6 * MIN_FRAME_SIZE);
-        assertFalse(set.contains(new ByteBufferPtr(buffer)));
-        buffer = pool.allocateFrame(1 * MIN_FRAME_SIZE);
-        assertTrue(set.contains(new ByteBufferPtr(buffer)));
-        assertEquals(5 * MIN_FRAME_SIZE, buffer.capacity());
-        pool.reset();
-    }
-
-    private void testAllocateShouldFailAfterAllSpaceGetUsed() throws HyracksDataException {
-        for (int i = 0; i < NUM_MIN_FRAME; i++) {
-            assertNull(pool.allocateFrame(MIN_FRAME_SIZE));
-        }
-    }
-
-    private HashSet<ByteBufferPtr> testAllocateAllSpacesWithMinFrames() throws HyracksDataException {
-        HashSet<ByteBufferPtr> set = new HashSet<>();
-        for (int i = 0; i < NUM_MIN_FRAME; i++) {
-            ByteBuffer buffer = pool.allocateFrame(MIN_FRAME_SIZE);
-            assertNotNull(buffer);
-            assertTrue(!set.contains(new ByteBufferPtr(buffer)));
-            set.add(new ByteBufferPtr(buffer));
-        }
-        return set;
-    }
-
-    /**
-     * Pool will become 1,2,3,4,5
-     *
-     * @throws HyracksDataException
-     */
-    private Set<ByteBufferPtr> testAllocateVariableFrames() throws HyracksDataException {
-        int budget = BUDGET;
-        int allocate = 0;
-        int i = 1;
-        Set<ByteBufferPtr> set = new HashSet<>();
-        while (budget - allocate >= i * MIN_FRAME_SIZE) {
-            ByteBuffer buffer = pool.allocateFrame(i * MIN_FRAME_SIZE);
-            assertNotNull(buffer);
-            set.add(new ByteBufferPtr(buffer));
-            allocate += i++ * MIN_FRAME_SIZE;
-        }
-        return set;
-    }
-
-    private void testShouldFindTheMatchFrames(Set<?> set) throws HyracksDataException {
-        pool.reset();
-        List<Integer> list = Arrays.asList(1, 2, 3, 4, 5);
-
-        for (int i = 0; i < list.size(); i++) {
-            ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
-            assertNotNull(buffer);
-            assertTrue(set.contains(new ByteBufferPtr(buffer)));
-            assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
-        }
-        pool.reset();
-        for (int i = list.size() - 1; i >= 0; i--) {
-            ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
-            assertNotNull(buffer);
-            assertTrue(set.contains(new ByteBufferPtr(buffer)));
-            assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
-        }
-
-        Collections.shuffle(list);
-        pool.reset();
-        for (int i = 0; i < list.size(); i++) {
-            ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
-            assertNotNull(buffer);
-            assertTrue(set.contains(new ByteBufferPtr(buffer)));
-            assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
-        }
-
-    }
-
-    public static class ByteBufferPtr {
-        ByteBuffer bytebuffer;
-
-        public ByteBufferPtr(ByteBuffer buffer) {
-            bytebuffer = buffer;
-        }
-
-        @Override
-        public int hashCode() {
-            return bytebuffer.hashCode();
-        }
-
-        @Override
-        public boolean equals(Object obj) {
-            return this.bytebuffer == ((ByteBufferPtr) obj).bytebuffer;
-        }
-    }
-
-}
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
deleted file mode 100644
index d14e15b..0000000
--- a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
+++ /dev/null
@@ -1,234 +0,0 @@
-/*
- * 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 org.apache.hyracks.dataflow.std.sort.buffermanager;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Random;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.hyracks.api.comm.FixedSizeFrame;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.util.IntSerDeUtils;
-import org.apache.hyracks.dataflow.std.sort.Utility;
-import org.apache.hyracks.dataflow.std.structures.TuplePointer;
-
-public class VariableTupleMemoryManagerTest {
-    ISerializerDeserializer[] fieldsSerDer = new ISerializerDeserializer[] {
-            IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer() };
-    RecordDescriptor recordDescriptor = new RecordDescriptor(fieldsSerDer);
-    ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
-    VariableTupleMemoryManager tupleMemoryManager;
-    FrameTupleAccessor inFTA = new FrameTupleAccessor(recordDescriptor);
-    Random random = new Random(System.currentTimeMillis());
-
-    @Before
-    public void setup() {
-        VariableFramePool framePool = new VariableFramePool(Common.commonFrameManager, Common.BUDGET);
-        tupleMemoryManager = new VariableTupleMemoryManager(framePool, recordDescriptor);
-    }
-
-    @Test
-    public void testInsertTupleToMemoryManager() throws HyracksDataException {
-        int iTuplePerFrame = 3;
-        Map<Integer, Integer> mapPrepare = prepareFixedSizeTuples(iTuplePerFrame);
-        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
-        assertEachTupleInFTAIsInBuffer(mapPrepare, mapInserted);
-    }
-
-    @Test
-    public void testReset() throws HyracksDataException {
-        testInsertVariableSizeTupleToMemoryManager();
-        tupleMemoryManager.reset();
-        testInsertTupleToMemoryManager();
-        tupleMemoryManager.reset();
-        testInsertVariableSizeTupleToMemoryManager();
-    }
-
-    @Test
-    public void testDeleteTupleInMemoryManager() throws HyracksDataException {
-        int iTuplePerFrame = 3;
-        Map<Integer, Integer> map = prepareFixedSizeTuples(iTuplePerFrame);
-        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
-        deleteRandomSelectedTuples(map, mapInserted, 1);
-        assertEachTupleInFTAIsInBuffer(map, mapInserted);
-    }
-
-    @Test
-    public void testReOrganizeSpace() throws HyracksDataException {
-        int iTuplePerFrame = 3;
-        Map<Integer, Integer> map = prepareFixedSizeTuples(iTuplePerFrame);
-        Map<Integer, Integer> copyMap = new HashMap<>(map);
-        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
-        ByteBuffer buffer = deleteRandomSelectedTuples(map, mapInserted, map.size() / 2);
-        inFTA.reset(buffer);
-        Map<TuplePointer, Integer> mapInserted2 = insertInFTAToBufferShouldAllSuccess();
-        Map<TuplePointer, Integer> mergedMap = new HashMap<>(mapInserted);
-        mergedMap.putAll(mapInserted2);
-        assertEachTupleInFTAIsInBuffer(copyMap, mergedMap);
-    }
-
-    @Test
-    public void testReOrganizeVariableSizeTuple() throws HyracksDataException {
-        Map<Integer, Integer> map = prepareVariableSizeTuples();
-        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferCouldFailForLargerTuples(map);
-        Map<Integer, Integer> copyMap = new HashMap<>(map);
-
-        ByteBuffer buffer = deleteRandomSelectedTuples(map, mapInserted, map.size() / 2);
-        inFTA.reset(buffer);
-
-        Map<TuplePointer, Integer> mapInserted2 = insertInFTAToBufferCouldFailForLargerTuples(copyMap);
-        Map<TuplePointer, Integer> mergedMap = new HashMap<>(mapInserted);
-        mergedMap.putAll(mapInserted2);
-
-        assertEachTupleInFTAIsInBuffer(copyMap, mergedMap);
-    }
-
-    @Test
-    public void testInsertVariableSizeTupleToMemoryManager() throws HyracksDataException {
-        Map<Integer, Integer> map = prepareVariableSizeTuples();
-        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferCouldFailForLargerTuples(map);
-        assertEachTupleInFTAIsInBuffer(map, mapInserted);
-    }
-
-    private void assertEachTupleInFTAIsInBuffer(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted) {
-        ITupleBufferAccessor accessor = tupleMemoryManager.getTupleAccessor();
-        for (Map.Entry<TuplePointer, Integer> entry : mapInserted.entrySet()) {
-            accessor.reset(entry.getKey());
-            int dataLength = map.get(entry.getValue());
-            assertEquals((int) entry.getValue(),
-                    IntSerDeUtils.getInt(accessor.getTupleBuffer().array(), accessor.getAbsFieldStartOffset(0)));
-            assertEquals(dataLength, accessor.getTupleLength());
-        }
-        assertEquals(map.size(), mapInserted.size());
-    }
-
-    private Map<Integer, Integer> prepareFixedSizeTuples(int tuplePerFrame) throws HyracksDataException {
-        Map<Integer, Integer> dataSet = new HashMap<>();
-        ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
-        FixedSizeFrame frame = new FixedSizeFrame(buffer);
-        FrameTupleAppender appender = new FrameTupleAppender();
-        appender.reset(frame, true);
-
-        int sizePerTuple = (Common.MIN_FRAME_SIZE - 1 - 4 - tuplePerFrame * 4 - 4) / tuplePerFrame;
-        int sizeChar = sizePerTuple - fieldsSerDer.length * 4 - 4 - 4;
-        assert (sizeChar > 0);
-        for (int i = 0; i < Common.NUM_MIN_FRAME * tuplePerFrame; i++) {
-            tupleBuilder.reset();
-            tupleBuilder.addField(fieldsSerDer[0], i);
-            tupleBuilder.addField(fieldsSerDer[1], Utility.repeatString('a', sizeChar));
-            assertTrue(appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
-                    tupleBuilder.getSize()));
-            dataSet.put(i, tupleBuilder.getSize() + tupleBuilder.getFieldEndOffsets().length * 4);
-        }
-        inFTA.reset(buffer);
-        return dataSet;
-    }
-
-    private Map<Integer, Integer> prepareVariableSizeTuples() throws HyracksDataException {
-        Map<Integer, Integer> dataSet = new HashMap<>();
-        ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
-        FixedSizeFrame frame = new FixedSizeFrame(buffer);
-        FrameTupleAppender appender = new FrameTupleAppender();
-        appender.reset(frame, true);
-
-        for (int i = 0; true; i++) {
-            tupleBuilder.reset();
-            tupleBuilder.addField(fieldsSerDer[0], i);
-            tupleBuilder.addField(fieldsSerDer[1], Utility.repeatString('a', i));
-            if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
-                    tupleBuilder.getSize())) {
-                break;
-            }
-            dataSet.put(i, tupleBuilder.getSize() + tupleBuilder.getFieldEndOffsets().length * 4);
-        }
-        inFTA.reset(buffer);
-        return dataSet;
-    }
-
-    private Map<TuplePointer, Integer> insertInFTAToBufferShouldAllSuccess() throws HyracksDataException {
-        Map<TuplePointer, Integer> tuplePointerIntegerMap = new HashMap<>();
-        for (int i = 0; i < inFTA.getTupleCount(); i++) {
-            TuplePointer tuplePointer = new TuplePointer();
-            assertTrue(tupleMemoryManager.insertTuple(inFTA, i, tuplePointer));
-            tuplePointerIntegerMap.put(tuplePointer,
-                    IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
-        }
-        return tuplePointerIntegerMap;
-    }
-
-    private Map<TuplePointer, Integer> insertInFTAToBufferCouldFailForLargerTuples(Map<Integer, Integer> map)
-            throws HyracksDataException {
-        Map<TuplePointer, Integer> tuplePointerIdMap = new HashMap<>();
-        int i = 0;
-        for (; i < inFTA.getTupleCount(); i++) {
-            TuplePointer tuplePointer = new TuplePointer();
-            if (!tupleMemoryManager.insertTuple(inFTA, i, tuplePointer)) {
-                break;
-            }
-            tuplePointerIdMap.put(tuplePointer,
-                    IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
-        }
-        for (; i < inFTA.getTupleCount(); i++) {
-            map.remove(IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
-        }
-        return tuplePointerIdMap;
-    }
-
-    private ByteBuffer deleteRandomSelectedTuples(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted,
-            int minNumOfRecordTobeDeleted)
-            throws HyracksDataException {
-        ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
-        FixedSizeFrame frame = new FixedSizeFrame(buffer);
-        FrameTupleAppender appender = new FrameTupleAppender();
-        appender.reset(frame, true);
-
-        assert (minNumOfRecordTobeDeleted < mapInserted.size());
-        int countDeleted = minNumOfRecordTobeDeleted + random.nextInt(mapInserted.size() - minNumOfRecordTobeDeleted);
-
-        ITupleBufferAccessor accessor = tupleMemoryManager.getTupleAccessor();
-        for (int i = 0; i < countDeleted; i++) {
-            Iterator<Map.Entry<TuplePointer, Integer>> iter = mapInserted.entrySet().iterator();
-            assert (iter.hasNext());
-            Map.Entry<TuplePointer, Integer> pair = iter.next();
-            accessor.reset(pair.getKey());
-            appender.append(accessor.getTupleBuffer().array(), accessor.getTupleStartOffset(),
-                    accessor.getTupleLength());
-            map.remove(pair.getValue());
-            tupleMemoryManager.deleteTuple(pair.getKey());
-            iter.remove();
-        }
-        return buffer;
-    }
-}
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTableTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTableTest.java
new file mode 100644
index 0000000..5893f35
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTableTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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 org.apache.hyracks.dataflow.std.structures;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.control.nc.resources.memory.FrameManager;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SerializableHashTableTest {
+
+    SerializableHashTable nsTable;
+    final int NUM_PART = 101;
+    TuplePointer pointer = new TuplePointer(0, 0);
+    final int num = 1000;
+
+    @Before
+    public void setup() throws HyracksDataException {
+        nsTable = new SerializableHashTable(NUM_PART, new FrameManager(256));
+    }
+
+    @Test
+    public void testBatchDeletePartition() throws Exception {
+        testInsert();
+        for (int i = 0; i < NUM_PART; i++) {
+            nsTable.delete(i);
+            assertFalse(nsTable.getTuplePointer(i, 0, pointer));
+            assertEquals(0, nsTable.getTupleCount(i));
+
+            for (int j = i; j < num; j += NUM_PART) {
+                pointer.reset(j, j);
+                nsTable.insert(i, pointer);
+            }
+
+            assertGetValue();
+        }
+    }
+
+    @Test
+    public void testInsert() throws Exception {
+        for (int i = 0; i < num; i++) {
+            pointer.reset(i, i);
+            nsTable.insert(i % NUM_PART, pointer);
+        }
+        assertGetValue();
+    }
+
+    private void assertGetValue() {
+        int loop = 0;
+        for (int i = 0; i < num; i++) {
+            assertTrue(nsTable.getTuplePointer(i % NUM_PART, loop, pointer));
+            assertTrue(pointer.frameIndex == i);
+            if (i % NUM_PART == NUM_PART - 1) {
+                loop++;
+            }
+        }
+        for (int i = 0; i < NUM_PART; i++) {
+            assertTrue(nsTable.getTupleCount(i) == 10 || nsTable.getTupleCount(i) == 9);
+        }
+
+    }
+
+    @Test
+    public void testGetCount() throws Exception {
+        assertAllPartitionsCountIsZero();
+    }
+
+    private void assertAllPartitionsCountIsZero() {
+        for (int i = 0; i < NUM_PART; i++) {
+            assertEquals(0, nsTable.getTupleCount(i));
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AggregationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AggregationTest.java
index c330f8e..e3093c4 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AggregationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/AggregationTest.java
@@ -21,12 +21,11 @@
 import java.io.File;
 import java.io.IOException;
 
-import org.junit.Test;
-
 import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.dataset.ResultSetId;
@@ -34,6 +33,7 @@
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import org.apache.hyracks.data.std.accessors.UTF8StringBinaryHashFunctionFamily;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
@@ -63,18 +63,18 @@
 import org.apache.hyracks.dataflow.std.group.aggregators.MinMaxStringFieldAggregatorFactory;
 import org.apache.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
 import org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import org.apache.hyracks.tests.util.ResultSerializerFactoryProvider;
+import org.junit.Test;
 
 /**
  *
  */
 public class AggregationTest extends AbstractIntegrationTest {
 
-    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC2_ID,
-            new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
+    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(
+            new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
 
     final RecordDescriptor desc = new RecordDescriptor(new ISerializerDeserializer[] {
             new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE,
@@ -106,50 +106,6 @@
     }
 
     @Test
-    public void singleKeySumInmemGroupTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
-                desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 0 };
-        int tableSize = 8;
-
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
-                        new FloatSumFieldAggregatorFactory(5, true) }), outputRec, tableSize);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
-        spec.connect(conn1, csvScanner, 0, grouper, 0);
-
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
-        spec.connect(conn2, grouper, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @Test
     public void singleKeySumPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
@@ -168,14 +124,14 @@
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
                         new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
-                        new FloatSumFieldAggregatorFactory(5, true) }), outputRec);
+                        new FloatSumFieldAggregatorFactory(5, true) }),
+                outputRec);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
@@ -205,26 +161,26 @@
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
+        long fileSize = frameLimits * spec.getFrameSize();
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, tableSize, fileSize,
+                keyFields, frameLimits,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
-                                new IntSumFieldAggregatorFactory(3, false),
-                                new FloatSumFieldAggregatorFactory(5, false) }), new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
-                                new IntSumFieldAggregatorFactory(2, false),
-                                new FloatSumFieldAggregatorFactory(3, false) }), outputRec,
-                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
+                new UTF8StringNormalizedKeyComputerFactory(),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(3, false),
+                        new FloatSumFieldAggregatorFactory(5, false) }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(2, false),
+                        new FloatSumFieldAggregatorFactory(3, false) }),
+                outputRec, outputRec, new HashSpillableTableFactory(
+                        new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE }));
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumExtGroupTest");
@@ -239,50 +195,6 @@
     }
 
     @Test
-    public void singleKeyAvgInmemGroupTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
-                desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 0 };
-        int tableSize = 8;
-
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
-                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
-        spec.connect(conn1, csvScanner, 0, grouper, 0);
-
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
-        spec.connect(conn2, grouper, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @Test
     public void singleKeyAvgPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
@@ -299,16 +211,16 @@
 
         PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
-                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, true),
+                                new CountFieldAggregatorFactory(true), new AvgFieldGroupAggregatorFactory(1, true) }),
+                outputRec);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
@@ -338,26 +250,26 @@
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
+        long fileSize = frameLimits * spec.getFrameSize();
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, tableSize, fileSize,
+                keyFields, frameLimits,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new UTF8StringNormalizedKeyComputerFactory(),
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
-                        new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+                new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
-                                new IntSumFieldAggregatorFactory(2, false),
-                                new AvgFieldMergeAggregatorFactory(3, false) }), outputRec,
-                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
+                                new CountFieldAggregatorFactory(false), new AvgFieldGroupAggregatorFactory(1, false) }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(2, false),
+                        new AvgFieldMergeAggregatorFactory(3, false) }),
+                outputRec, outputRec, new HashSpillableTableFactory(
+                        new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE }));
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
@@ -372,50 +284,6 @@
     }
 
     @Test
-    public void singleKeyMinMaxStringInmemGroupTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
-                desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE,
-                new UTF8StringSerializerDeserializer() });
-
-        int[] keyFields = new int[] { 0 };
-        int tableSize = 8;
-
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true),
-                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
-        spec.connect(conn1, csvScanner, 0, grouper, 0);
-
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
-        spec.connect(conn2, grouper, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @Test
     public void singleKeyMinMaxStringPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
@@ -424,24 +292,24 @@
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE,
-                new UTF8StringSerializerDeserializer() });
+        RecordDescriptor outputRec = new RecordDescriptor(
+                new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer(),
+                        IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer() });
 
         int[] keyFields = new int[] { 0 };
 
         PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true),
-                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, true),
+                                new MinMaxStringFieldAggregatorFactory(15, true, false) }),
+                outputRec);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
@@ -464,80 +332,36 @@
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE,
-                new UTF8StringSerializerDeserializer() });
+        RecordDescriptor outputRec = new RecordDescriptor(
+                new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer(),
+                        IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer() });
 
         int[] keyFields = new int[] { 0 };
         int frameLimits = 4;
         int tableSize = 8;
+        long fileSize = frameLimits * spec.getFrameSize();
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, tableSize, fileSize,
+                keyFields, frameLimits,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                new UTF8StringNormalizedKeyComputerFactory(),
+                new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
                                 new MinMaxStringFieldAggregatorFactory(15, true, true) }),
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, false),
-                        new MinMaxStringFieldAggregatorFactory(2, true, true) }), outputRec,
-                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }), tableSize), true);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
-        spec.connect(conn1, csvScanner, 0, grouper, 0);
-
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
-        spec.connect(conn2, grouper, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @Test
-    public void multiKeySumInmemGroupTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
-                desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 8, 0 };
-        int tableSize = 8;
-
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
-                outputRec, tableSize);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new MinMaxStringFieldAggregatorFactory(2, true, true) }),
+                outputRec, outputRec, new HashSpillableTableFactory(
+                        new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE }));
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
                         PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
 
@@ -573,9 +397,10 @@
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] {
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
@@ -605,26 +430,27 @@
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
+        long fileSize = frameLimits * spec.getFrameSize();
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, tableSize, fileSize,
+                keyFields, frameLimits,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
-                                new IntSumFieldAggregatorFactory(3, false) }), new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
-                                new IntSumFieldAggregatorFactory(3, false) }), outputRec,
-                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+                new UTF8StringNormalizedKeyComputerFactory(),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(3, false) }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(2, false), new IntSumFieldAggregatorFactory(3, false) }),
+                outputRec, outputRec, new HashSpillableTableFactory(new IBinaryHashFunctionFamily[] {
+                        UTF8StringBinaryHashFunctionFamily.INSTANCE, UTF8StringBinaryHashFunctionFamily.INSTANCE }));
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] {
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumExtGroupTest");
@@ -639,52 +465,6 @@
     }
 
     @Test
-    public void multiKeyAvgInmemGroupTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
-                desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
-                FloatSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 8, 0 };
-        int tableSize = 8;
-
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
-                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
-        spec.connect(conn1, csvScanner, 0, grouper, 0);
-
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
-        spec.connect(conn2, grouper, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @Test
     public void multiKeyAvgPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
@@ -693,26 +473,28 @@
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
-                FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(
+                new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer(),
+                        new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE,
+                        IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
 
         PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
-                        new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, true),
+                                new CountFieldAggregatorFactory(true), new AvgFieldGroupAggregatorFactory(1, true) }),
+                outputRec);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] {
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
@@ -735,36 +517,37 @@
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
 
-        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
-                FloatSerializerDeserializer.INSTANCE });
+        RecordDescriptor outputRec = new RecordDescriptor(
+                new ISerializerDeserializer[] { new UTF8StringSerializerDeserializer(),
+                        new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE,
+                        IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
 
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
+        long fileSize = frameLimits * spec.getFrameSize();
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, tableSize, fileSize,
+                keyFields, frameLimits,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 new UTF8StringNormalizedKeyComputerFactory(),
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new CountFieldAggregatorFactory(false), new AvgFieldGroupAggregatorFactory(1, false) }),
                 new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
-                        new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
-                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
-                                new IntSumFieldAggregatorFactory(3, false),
-                                new AvgFieldMergeAggregatorFactory(4, false) }), outputRec,
-                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+                        new IntSumFieldAggregatorFactory(2, false), new IntSumFieldAggregatorFactory(3, false),
+                        new AvgFieldMergeAggregatorFactory(4, false) }),
+                outputRec, outputRec, new HashSpillableTableFactory(new IBinaryHashFunctionFamily[] {
+                        UTF8StringBinaryHashFunctionFamily.INSTANCE, UTF8StringBinaryHashFunctionFamily.INSTANCE }));
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] {
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgExtGroupTest");
@@ -779,51 +562,6 @@
     }
 
     @Test
-    public void multiKeyMinMaxStringInmemGroupTest() throws Exception {
-        JobSpecification spec = new JobSpecification();
-
-        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
-                desc);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
-
-        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer() });
-
-        int[] keyFields = new int[] { 8, 0 };
-        int tableSize = 8;
-
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true),
-                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
-        spec.connect(conn1, csvScanner, 0, grouper, 0);
-
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringInmemGroupTest");
-
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
-
-        IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
-        spec.connect(conn2, grouper, 0, printer, 0);
-
-        spec.addRoot(printer);
-        runTest(spec);
-    }
-
-    @Test
     public void multiKeyMinMaxStringPreClusterGroupTest() throws Exception {
         JobSpecification spec = new JobSpecification();
 
@@ -841,16 +579,18 @@
         PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true),
-                        new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
+                new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, true),
+                                new MinMaxStringFieldAggregatorFactory(15, true, false) }),
+                outputRec);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] {
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringPreClusterGroupTest");
@@ -880,27 +620,29 @@
         int[] keyFields = new int[] { 8, 0 };
         int frameLimits = 4;
         int tableSize = 8;
+        long fileSize = frameLimits * spec.getFrameSize();
 
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, tableSize, fileSize,
+                keyFields, frameLimits,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
                         PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                new UTF8StringNormalizedKeyComputerFactory(),
+                new MultiFieldsAggregatorFactory(
                         new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
                                 new MinMaxStringFieldAggregatorFactory(15, true, true) }),
-                new MultiFieldsAggregatorFactory(new int[] { 0, 1 }, new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(2, false),
-                        new MinMaxStringFieldAggregatorFactory(3, true, true) }), outputRec,
-                new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+                new MultiFieldsAggregatorFactory(new int[] { 0, 1 },
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
+                                new MinMaxStringFieldAggregatorFactory(3, true, true) }),
+                outputRec, outputRec, new HashSpillableTableFactory(new IBinaryHashFunctionFamily[] {
+                        UTF8StringBinaryHashFunctionFamily.INSTANCE, UTF8StringBinaryHashFunctionFamily.INSTANCE }));
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
 
         IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(keyFields,
+                        new IBinaryHashFunctionFactory[] {
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
         AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringExtGroupTest");
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/OptimizedSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/HeapSortMergeTest.java
similarity index 99%
rename from hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/OptimizedSortMergeTest.java
rename to hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/HeapSortMergeTest.java
index 555694f..699389b 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/OptimizedSortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/HeapSortMergeTest.java
@@ -52,7 +52,7 @@
 import org.apache.hyracks.dataflow.std.sort.TopKSorterOperatorDescriptor;
 import org.apache.hyracks.tests.util.ResultSerializerFactoryProvider;
 
-public class OptimizedSortMergeTest extends AbstractIntegrationTest {
+public class HeapSortMergeTest extends AbstractIntegrationTest {
 
     @Test
     public void optimizedSortMergeTest01() throws Exception {
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/LocalityAwareConnectorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/LocalityAwareConnectorTest.java
index 27650e4..ad3c006 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/LocalityAwareConnectorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/LocalityAwareConnectorTest.java
@@ -22,12 +22,11 @@
 import java.io.IOException;
 import java.util.BitSet;
 
-import org.junit.Test;
-
 import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.dataset.ResultSetId;
@@ -35,10 +34,12 @@
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import org.apache.hyracks.data.std.accessors.UTF8StringBinaryHashFunctionFamily;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
@@ -56,21 +57,25 @@
 import org.apache.hyracks.dataflow.std.file.FileSplit;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
+import org.apache.hyracks.dataflow.std.group.HashSpillableTableFactory;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.aggregators.FloatSumFieldAggregatorFactory;
 import org.apache.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
 import org.apache.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
-import org.apache.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
 import org.apache.hyracks.tests.util.ResultSerializerFactoryProvider;
+import org.junit.Test;
 
 public class LocalityAwareConnectorTest extends AbstractMultiNCIntegrationTest {
 
-    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] {
-            new FileSplit("asterix-001", new FileReference(new File("data/tpch0.001/lineitem.tbl"))),
-            new FileSplit("asterix-002", new FileReference(new File("data/tpch0.001/lineitem.tbl"))),
-            new FileSplit("asterix-003", new FileReference(new File("data/tpch0.001/lineitem.tbl"))),
-            new FileSplit("asterix-004", new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
+    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(
+            new FileSplit[] { new FileSplit("asterix-001", new FileReference(new File("data/tpch0.001/lineitem.tbl"))),
+                    new FileSplit("asterix-002", new FileReference(new File("data/tpch0.001/lineitem.tbl"))),
+                    new FileSplit("asterix-003", new FileReference(new File("data/tpch0.001/lineitem.tbl"))),
+                    new FileSplit("asterix-004", new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
+
+    final int fileSize = 800 * 1024 * 4;
 
     final RecordDescriptor desc = new RecordDescriptor(new ISerializerDeserializer[] {
             new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE,
@@ -116,14 +121,18 @@
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }),
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, tableSize, fileSize,
+                keyFields, fileSize / spec.getFrameSize() + 1,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(),
                 new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
-                        new FloatSumFieldAggregatorFactory(5, true) }), outputRec, tableSize);
+                        new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(3, false),
+                        new FloatSumFieldAggregatorFactory(5, false) }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(2, false),
+                        new FloatSumFieldAggregatorFactory(3, false) }),
+                outputRec, outputRec, new HashSpillableTableFactory(
+                        new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE }));
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, "asterix-005", "asterix-006");
 
@@ -136,8 +145,9 @@
 
         IConnectorDescriptor conn1 = new LocalityAwareMToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }), new HashtableLocalityMap(nodemap));
+                        new IBinaryHashFunctionFactory[] {
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new HashtableLocalityMap(nodemap));
 
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
@@ -176,21 +186,26 @@
         int[] keyFields = new int[] { 0 };
         int tableSize = 8;
 
-        HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
-                new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }),
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, tableSize, fileSize,
+                keyFields, fileSize / spec.getFrameSize() + 1,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(),
                 new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
-                        new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
-                        new FloatSumFieldAggregatorFactory(5, true) }), outputRec, tableSize);
+                        new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(3, false),
+                        new FloatSumFieldAggregatorFactory(5, false) }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                        new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(2, false),
+                        new FloatSumFieldAggregatorFactory(3, false) }),
+                outputRec, outputRec, new HashSpillableTableFactory(
+                        new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE }));
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, "asterix-005", "asterix-006");
 
         IConnectorDescriptor conn1 = new LocalityAwareMToNPartitioningConnectorDescriptor(spec,
                 new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }), new GlobalHashingLocalityMap());
+                        new IBinaryHashFunctionFactory[] {
+                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+                new GlobalHashingLocalityMap());
 
         spec.connect(conn1, csvScanner, 0, grouper, 0);
 
@@ -209,7 +224,7 @@
             throws IOException {
 
         ResultSetId rsId = new ResultSetId(1);
-        AbstractSingleActivityOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,false,
+        AbstractSingleActivityOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true, false,
                 ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
         spec.addResultSetId(rsId);
 
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOptimizedHybridHashJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOptimizedHybridHashJoinTest.java
index a10513a..a4c87c8 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOptimizedHybridHashJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOptimizedHybridHashJoinTest.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.tests.integration;
 
 import java.io.File;
+import java.util.Arrays;
 
 import org.junit.Test;
 
@@ -43,63 +44,73 @@
 import org.apache.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.file.FileSplit;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.join.JoinComparatorFactory;
 import org.apache.hyracks.dataflow.std.join.OptimizedHybridHashJoinOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
 
 public class TPCHCustomerOptimizedHybridHashJoinTest extends AbstractIntegrationTest {
-    private static final boolean DEBUG = false;
+
+    private static boolean DEBUG = false;
+
+    static RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() });
+
+    static RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer() });
+
+    static RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer() });
+
+    static IValueParserFactory[] custValueParserFactories = new IValueParserFactory[custDesc.getFieldCount()];
+    static IValueParserFactory[] orderValueParserFactories = new IValueParserFactory[ordersDesc.getFieldCount()];
+
+    static {
+        Arrays.fill(custValueParserFactories, UTF8StringParserFactory.INSTANCE);
+        Arrays.fill(orderValueParserFactories, UTF8StringParserFactory.INSTANCE);
+    }
+
+    private IOperatorDescriptor getPrinter(JobSpecification spec, File file) {
+        IFileSplitProvider outputSplitProvider = new ConstantFileSplitProvider(
+                new FileSplit[] {
+                        new FileSplit(NC1_ID, file.getAbsolutePath()) });
+
+        return DEBUG ? new PlainFileWriterOperatorDescriptor(spec, outputSplitProvider, "|")
+                : new NullSinkOperatorDescriptor(spec);
+    }
 
     @Test
     public void customerOrderCIDHybridHashJoin_Case1() throws Exception {
         JobSpecification spec = new JobSpecification();
-
         FileSplit[] custSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
                 "data/tpch0.001/customer4.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
-        RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() });
 
         FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
                 "data/tpch0.001/orders4.tbl"))) };
 
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer() });
-
-        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer() });
-
         FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+                new DelimitedDataTupleParserFactory(orderValueParserFactories, '|'), ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
+                new DelimitedDataTupleParserFactory(custValueParserFactories, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
         OptimizedHybridHashJoinOperatorDescriptor join = new OptimizedHybridHashJoinOperatorDescriptor(spec, 15, 243,
@@ -107,13 +118,14 @@
                 new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 custOrderJoinDesc, new JoinComparatorFactory(
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
-                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), null);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
+                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0),
+                null);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
-                : new NullSinkOperatorDescriptor(spec);
+        File file = File.createTempFile(getClass().getName(), "case1");
+        IOperatorDescriptor printer = getPrinter(spec, file);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor custJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -127,6 +139,7 @@
 
         spec.addRoot(printer);
         runTest(spec);
+        System.out.println("output to " + file.getAbsolutePath());
     }
 
     @Test
@@ -136,48 +149,18 @@
         FileSplit[] custSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
                 "data/tpch0.001/customer3.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
-        RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() });
 
         FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
                 "data/tpch0.001/orders4.tbl"))) };
 
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer() });
-
-        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer() });
 
         FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+                new DelimitedDataTupleParserFactory(orderValueParserFactories, '|'), ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
+                new DelimitedDataTupleParserFactory(custValueParserFactories, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
         OptimizedHybridHashJoinOperatorDescriptor join = new OptimizedHybridHashJoinOperatorDescriptor(spec, 15, 122,
@@ -185,13 +168,14 @@
                 new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 custOrderJoinDesc, new JoinComparatorFactory(
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
-                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), null);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
+                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0),
+                null);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
-                : new NullSinkOperatorDescriptor(spec);
+        File file = File.createTempFile(getClass().getName(), "case2");
+        IOperatorDescriptor printer = getPrinter(spec, file);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor custJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -205,6 +189,7 @@
 
         spec.addRoot(printer);
         runTest(spec);
+        System.out.println("output to " + file.getAbsolutePath());
     }
 
     @Test
@@ -215,48 +200,18 @@
         FileSplit[] custSplits = new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(
                 "data/tpch0.001/customer3.tbl"))) };
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
-        RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() });
 
         FileSplit[] ordersSplits = new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
                 "data/tpch0.001/orders1.tbl"))) };
 
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer() });
-
-        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer() });
 
         FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+                new DelimitedDataTupleParserFactory(orderValueParserFactories, '|'), ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
+                new DelimitedDataTupleParserFactory(custValueParserFactories, '|'), custDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
 
         OptimizedHybridHashJoinOperatorDescriptor join = new OptimizedHybridHashJoinOperatorDescriptor(spec, 6, 122,
@@ -264,13 +219,14 @@
                 new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                 custOrderJoinDesc, new JoinComparatorFactory(
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
-                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), null);
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
+                new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0),
+                null);
 
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
-        IOperatorDescriptor printer = DEBUG ? new PrinterOperatorDescriptor(spec)
-                : new NullSinkOperatorDescriptor(spec);
+        File file = File.createTempFile(getClass().getName(), "case3");
+        IOperatorDescriptor printer = getPrinter(spec, file);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         IConnectorDescriptor custJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -284,6 +240,7 @@
 
         spec.addRoot(printer);
         runTest(spec);
+        System.out.println("output to " + file.getAbsolutePath());
     }
 
 }
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index 2650799..8232a62 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -292,7 +292,7 @@
                 new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc, null);
+                custOrderJoinDesc, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
 
         ResultSetId rsId = new ResultSetId(1);
@@ -815,7 +815,7 @@
                 new int[] { 0 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                custOrderJoinDesc, null);
+                custOrderJoinDesc, null, false, null);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
 
         ResultSetId rsId = new ResultSetId(1);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/VSizeFrameSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/VSizeFrameSortMergeTest.java
index b774e0e..039497c 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/VSizeFrameSortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/VSizeFrameSortMergeTest.java
@@ -87,7 +87,6 @@
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
-        //                PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID );
 
         spec.setFrameSize(frameSize);
         ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, frameLimit, new int[] { 1, 0 },
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractExternalGroupbyTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractExternalGroupbyTest.java
new file mode 100644
index 0000000..b8ec790
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractExternalGroupbyTest.java
@@ -0,0 +1,243 @@
+/*
+ * 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 org.apache.hyracks.tests.unit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import org.apache.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
+import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.AvgFieldGroupAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.AvgFieldMergeAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
+import org.apache.hyracks.test.support.TestUtils;
+
+public abstract class AbstractExternalGroupbyTest {
+
+    ISerializerDeserializer[] inFields = new ISerializerDeserializer[] {
+            IntegerSerializerDeserializer.INSTANCE,
+            new UTF8StringSerializerDeserializer(),
+    };
+
+    ISerializerDeserializer[] aggrFields = new ISerializerDeserializer[] {
+            new UTF8StringSerializerDeserializer(),  // key
+            IntegerSerializerDeserializer.INSTANCE,     // sum
+            IntegerSerializerDeserializer.INSTANCE,     // count
+            FloatSerializerDeserializer.INSTANCE,       // avg
+    };
+
+    RecordDescriptor inRecordDesc = new RecordDescriptor(inFields);
+
+    RecordDescriptor outputRec = new RecordDescriptor(aggrFields);
+
+    IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[] {
+            PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
+
+    INormalizedKeyComputerFactory normalizedKeyComputerFactory = new UTF8StringNormalizedKeyComputerFactory();
+
+    IAggregatorDescriptorFactory partialAggrInPlace = new MultiFieldsAggregatorFactory(
+            new IFieldAggregateDescriptorFactory[] {
+                    new IntSumFieldAggregatorFactory(0, false),
+                    new CountFieldAggregatorFactory(false),
+                    new AvgFieldGroupAggregatorFactory(0, false) });
+
+    IAggregatorDescriptorFactory finalAggrInPlace = new MultiFieldsAggregatorFactory(
+            new IFieldAggregateDescriptorFactory[] {
+                    new IntSumFieldAggregatorFactory(1, false),
+                    new IntSumFieldAggregatorFactory(2, false),
+                    new AvgFieldMergeAggregatorFactory(3, false) });
+
+    IAggregatorDescriptorFactory partialAggrInState = new MultiFieldsAggregatorFactory(
+            new IFieldAggregateDescriptorFactory[] {
+                    new IntSumFieldAggregatorFactory(0, true),
+                    new CountFieldAggregatorFactory(true),
+                    new AvgFieldGroupAggregatorFactory(0, true) });
+
+    IAggregatorDescriptorFactory finalAggrInState = new MultiFieldsAggregatorFactory(
+            new IFieldAggregateDescriptorFactory[] {
+                    new IntSumFieldAggregatorFactory(1, true),
+                    new IntSumFieldAggregatorFactory(2, true),
+                    new AvgFieldMergeAggregatorFactory(3, true) });
+
+    int[] keyFields = new int[] { 1 };
+    int[] keyFieldsAfterPartial = new int[] { 0 };
+
+    class ResultValidateWriter implements IFrameWriter {
+
+        final Map<Integer, String> keyValueMap;
+        FrameTupleAccessor resultAccessor = new FrameTupleAccessor(outputRec);
+
+        class Result {
+            Result(int i) {
+                sum = i;
+                count = 1;
+            }
+
+            int sum;
+            int count;
+        }
+
+        private Map<String, Result> answer;
+
+        public ResultValidateWriter(Map<Integer, String> keyValueMap) {
+            this.keyValueMap = keyValueMap;
+            answer = new HashMap<>();
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            for (Map.Entry<Integer, String> keyValue : keyValueMap.entrySet()) {
+                Result result = answer.get(keyValue.getValue());
+                if (result == null) {
+                    answer.put(keyValue.getValue(), new Result(keyValue.getKey()));
+                } else {
+                    result.sum += keyValue.getKey();
+                    result.count++;
+                }
+            }
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            resultAccessor.reset(buffer);
+            ByteBufferInputStream bbis = new ByteBufferInputStream();
+            DataInputStream di = new DataInputStream(bbis);
+
+            Object[] outRecord = new Object[outputRec.getFieldCount()];
+
+            for (int tid = 0; tid < resultAccessor.getTupleCount(); tid++) {
+                for (int fid = 0; fid < outputRec.getFieldCount(); fid++) {
+                    bbis.setByteBuffer(resultAccessor.getBuffer(),
+                            resultAccessor.getAbsoluteFieldStartOffset(tid, fid));
+                    outRecord[fid] = outputRec.getFields()[fid].deserialize(di);
+                }
+                Result result = answer.remove((String) outRecord[0]);
+                assertNotNull(result);
+                assertEquals(result.sum, (int) outRecord[1]);
+                assertEquals(result.count, (int) outRecord[2]);
+            }
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+            Assert.fail();
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            assertEquals(0, answer.size());
+        }
+    }
+
+    @Test
+    public void testBuildAndMergeNormalFrameInMem() throws HyracksDataException {
+        int tableSize = 1001;
+        int numFrames = 3;
+        int frameSize = 256;
+        int minDataSize = frameSize;
+        int minRecordSize = 20;
+        int maxRecordSize = 50;
+        testBuildAndMerge(tableSize, numFrames, frameSize, minDataSize, minRecordSize, maxRecordSize, null);
+    }
+
+    @Test
+    public void testBuildAndMergeNormalFrameSpill() throws HyracksDataException {
+        int tableSize = 1001;
+        int numFrames = 3;
+        int frameSize = 256;
+        int minDataSize = frameSize * 4;
+        int minRecordSize = 20;
+        int maxRecordSize = 50;
+        testBuildAndMerge(tableSize, numFrames, frameSize, minDataSize, minRecordSize, maxRecordSize, null);
+    }
+
+    @Test
+    public void testBuildAndMergeBigObj() throws HyracksDataException {
+        int tableSize = 1001;
+        int numFrames = 4;
+        int frameSize = 256;
+        int minDataSize = frameSize * 5;
+        int minRecordSize = 20;
+        int maxRecordSize = 50;
+        HashMap<Integer, String> bigRecords = AbstractRunGeneratorTest.generateBigObject(frameSize, 2);
+        testBuildAndMerge(tableSize, numFrames, frameSize, minDataSize, minRecordSize, maxRecordSize,
+                bigRecords);
+
+    }
+
+    protected abstract void initial(IHyracksTaskContext ctx, int tableSize, int numFrames) throws HyracksDataException;
+
+    protected abstract IFrameWriter getBuilder();
+
+    protected abstract IOperatorNodePushable getMerger();
+
+    private void testBuildAndMerge(int tableSize, int numFrames, int frameSize, int minDataSize,
+            int minRecordSize, int maxRecordSize,
+            Map<Integer, String> specialData)
+            throws HyracksDataException {
+
+        IHyracksTaskContext ctx = TestUtils.create(frameSize);
+        initial(ctx, tableSize, numFrames);
+        ArrayList<IFrame> input = new ArrayList<>();
+        Map<Integer, String> keyValueMap = new HashMap<>();
+        AbstractRunGeneratorTest
+                .prepareData(ctx, input, minDataSize, minRecordSize, maxRecordSize, specialData, keyValueMap);
+
+        ResultValidateWriter writer = new ResultValidateWriter(keyValueMap);
+
+        getBuilder().open();
+        for (IFrame frame : input) {
+            getBuilder().nextFrame(frame.getBuffer());
+        }
+        getBuilder().close();
+
+        getMerger().setOutputFrameWriter(0, writer, outputRec);
+        getMerger().initialize();
+    }
+
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractRunGeneratorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractRunGeneratorTest.java
index 3cc2a23..673c6fa 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractRunGeneratorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/AbstractRunGeneratorTest.java
@@ -45,8 +45,8 @@
 import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
 import org.apache.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
-import org.apache.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
 import org.apache.hyracks.dataflow.std.sort.util.GroupFrameAccessor;
 import org.apache.hyracks.dataflow.std.sort.util.GroupVSizeFrame;
 import org.apache.hyracks.test.support.TestUtils;
@@ -54,8 +54,8 @@
 
 public abstract class AbstractRunGeneratorTest {
     static TestUtils testUtils = new TestUtils();
-    static ISerializerDeserializer[] SerDers = new ISerializerDeserializer[] {
-            IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer() };
+    static ISerializerDeserializer[] SerDers = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE,
+            new UTF8StringSerializerDeserializer() };
     static RecordDescriptor RecordDesc = new RecordDescriptor(SerDers);
     static Random GRandom = new Random(System.currentTimeMillis());
     static int[] SortFields = new int[] { 0, 1 };
@@ -63,17 +63,17 @@
             PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
             PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
 
-    static void assertMaxFrameSizesAreAllEqualsTo(List<RunAndMaxFrameSizePair> maxSize, int pageSize) {
+    static void assertMaxFrameSizesAreAllEqualsTo(List<GeneratedRunFileReader> maxSize, int pageSize) {
         for (int i = 0; i < maxSize.size(); i++) {
-            assertTrue(maxSize.get(i).maxFrameSize == pageSize);
+            assertTrue(maxSize.get(i).getMaxFrameSize() == pageSize);
         }
     }
 
     abstract AbstractSortRunGenerator getSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int numOfInputRecord)
             throws HyracksDataException;
 
-    protected List<RunAndMaxFrameSizePair> testSortRecords(int pageSize, int frameLimit, int numRuns,
-            int minRecordSize, int maxRecordSize, HashMap<Integer, String> specialData) throws HyracksDataException {
+    protected List<GeneratedRunFileReader> testSortRecords(int pageSize, int frameLimit, int numRuns, int minRecordSize,
+            int maxRecordSize, HashMap<Integer, String> specialData) throws HyracksDataException {
         IHyracksTaskContext ctx = testUtils.create(pageSize);
 
         HashMap<Integer, String> keyValuePair = new HashMap<>();
@@ -90,12 +90,12 @@
         return runGenerator.getRuns();
     }
 
-    static void matchResult(IHyracksTaskContext ctx, List<RunAndMaxFrameSizePair> runs,
+    static void matchResult(IHyracksTaskContext ctx, List<GeneratedRunFileReader> runs,
             Map<Integer, String> keyValuePair) throws HyracksDataException {
         HashMap<Integer, String> copyMap2 = new HashMap<>(keyValuePair);
         int maxFrameSizes = 0;
-        for (RunAndMaxFrameSizePair run : runs) {
-            maxFrameSizes = Math.max(maxFrameSizes, run.maxFrameSize);
+        for (GeneratedRunFileReader run : runs) {
+            maxFrameSizes = Math.max(maxFrameSizes, run.getMaxFrameSize());
         }
         GroupVSizeFrame gframe = new GroupVSizeFrame(ctx, maxFrameSizes);
         GroupFrameAccessor gfta = new GroupFrameAccessor(ctx.getInitialFrameSize(), RecordDesc);
@@ -125,25 +125,25 @@
         return preKey;
     }
 
-    static void assertReadSorted(List<RunAndMaxFrameSizePair> runs, IFrameTupleAccessor fta, IFrame frame,
+    static void assertReadSorted(List<GeneratedRunFileReader> runs, IFrameTupleAccessor fta, IFrame frame,
             Map<Integer, String> keyValuePair) throws HyracksDataException {
 
         assertTrue(runs.size() > 0);
-        for (RunAndMaxFrameSizePair run : runs) {
-            run.run.open();
+        for (GeneratedRunFileReader run : runs) {
+            run.open();
             int preKey = Integer.MIN_VALUE;
-            while (run.run.nextFrame(frame)) {
+            while (run.nextFrame(frame)) {
                 fta.reset(frame.getBuffer());
                 preKey = assertFTADataIsSorted(fta, keyValuePair, preKey);
             }
-            run.run.close();
+            run.close();
         }
         assertTrue(keyValuePair.isEmpty());
     }
 
     static void prepareData(IHyracksTaskContext ctx, List<IFrame> frameList, int minDataSize, int minRecordSize,
             int maxRecordSize, Map<Integer, String> specialData, Map<Integer, String> keyValuePair)
-            throws HyracksDataException {
+                    throws HyracksDataException {
 
         ArrayTupleBuilder tb = new ArrayTupleBuilder(RecordDesc.getFieldCount());
         FrameTupleAppender appender = new FrameTupleAppender();
@@ -223,7 +223,7 @@
         int numRuns = 2;
         int minRecordSize = pageSize / 8;
         int maxRecordSize = pageSize / 8;
-        List<RunAndMaxFrameSizePair> maxSize = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize,
+        List<GeneratedRunFileReader> maxSize = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize,
                 maxRecordSize, null);
         assertMaxFrameSizesAreAllEqualsTo(maxSize, pageSize);
     }
@@ -235,8 +235,8 @@
         int numRuns = 2;
         int minRecordSize = pageSize;
         int maxRecordSize = (int) (pageSize * 1.8);
-        List<RunAndMaxFrameSizePair> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize,
-                maxRecordSize, null);
+        List<GeneratedRunFileReader> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize, maxRecordSize,
+                null);
         assertMaxFrameSizesAreAllEqualsTo(size, pageSize * 2);
     }
 
@@ -248,12 +248,12 @@
         int minRecordSize = 20;
         int maxRecordSize = pageSize / 2;
         HashMap<Integer, String> specialPair = generateBigObject(pageSize, frameLimit - 1);
-        List<RunAndMaxFrameSizePair> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize,
-                maxRecordSize, specialPair);
+        List<GeneratedRunFileReader> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize, maxRecordSize,
+                specialPair);
 
         int max = 0;
-        for (RunAndMaxFrameSizePair run : size) {
-            max = Math.max(max, run.maxFrameSize);
+        for (GeneratedRunFileReader run : size) {
+            max = Math.max(max, run.getMaxFrameSize());
         }
         assertTrue(max == pageSize * (frameLimit - 1));
     }
@@ -266,8 +266,8 @@
         HashMap<Integer, String> specialPair = generateBigObject(pageSize, frameLimit);
         int minRecordSize = 10;
         int maxRecordSize = pageSize / 2;
-        List<RunAndMaxFrameSizePair> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize,
-                maxRecordSize, specialPair);
+        List<GeneratedRunFileReader> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize, maxRecordSize,
+                specialPair);
 
     }
 }
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/ExternalHashGroupbyTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/ExternalHashGroupbyTest.java
new file mode 100644
index 0000000..f1a4231
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/ExternalHashGroupbyTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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 org.apache.hyracks.tests.unit;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+import org.apache.hyracks.data.std.accessors.UTF8StringBinaryHashFunctionFamily;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.group.HashSpillableTableFactory;
+import org.apache.hyracks.dataflow.std.group.ISpillableTableFactory;
+import org.apache.hyracks.dataflow.std.group.external.ExternalGroupBuildOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.group.external.ExternalGroupWriteOperatorNodePushable;
+
+public class ExternalHashGroupbyTest extends AbstractExternalGroupbyTest {
+    ExternalGroupBuildOperatorNodePushable buildOperator;
+    ExternalGroupWriteOperatorNodePushable mergeOperator;
+
+    @Override
+    protected void initial(IHyracksTaskContext ctx, int tableSize, int numFrames) {
+        ISpillableTableFactory tableFactory = new HashSpillableTableFactory(
+                new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE });
+        buildOperator = new ExternalGroupBuildOperatorNodePushable(ctx, this.hashCode(), tableSize,
+                numFrames * ctx.getInitialFrameSize(), keyFields, numFrames, comparatorFactories,
+                normalizedKeyComputerFactory, partialAggrInPlace, inRecordDesc, outputRec, tableFactory);
+        mergeOperator = new ExternalGroupWriteOperatorNodePushable(ctx, this.hashCode(), tableFactory, outputRec,
+                outputRec, numFrames, keyFieldsAfterPartial, normalizedKeyComputerFactory, comparatorFactories,
+                finalAggrInPlace);
+    }
+
+    @Override
+    protected IFrameWriter getBuilder() {
+        return buildOperator;
+    }
+
+    @Override
+    protected AbstractUnaryOutputSourceOperatorNodePushable getMerger() {
+        return mergeOperator;
+    }
+
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/RunMergingFrameReaderTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/RunMergingFrameReaderTest.java
index e6d10f2..567b7df 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/RunMergingFrameReaderTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/RunMergingFrameReaderTest.java
@@ -55,10 +55,9 @@
 import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.io.RunFileReader;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
 import org.apache.hyracks.dataflow.std.sort.Algorithm;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
-import org.apache.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
 import org.apache.hyracks.dataflow.std.sort.RunMergingFrameReader;
 import org.apache.hyracks.dataflow.std.sort.util.GroupVSizeFrame;
 import org.junit.Test;
@@ -234,8 +233,8 @@
             List<Map<Integer, String>> keyValueMapList = new ArrayList<>(numRuns);
             List<TestFrameReader> readerList = new ArrayList<>(numRuns);
             List<IFrame> frameList = new ArrayList<>(numRuns);
-            prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun, minRecordSize, maxRecordSize,
-                    readerList, frameList, keyValueMapList);
+            prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun, minRecordSize, maxRecordSize, readerList,
+                    frameList, keyValueMapList);
 
             RunMergingFrameReader reader = new RunMergingFrameReader(ctx, readerList, frameList, SortFields,
                     Comparators, null, RecordDesc, topK);
@@ -313,8 +312,8 @@
         int maxRecordSize = pageSize / 2;
 
         IHyracksTaskContext ctx = testUtils.create(pageSize);
-        ExternalSortRunGenerator runGenerator = new ExternalSortRunGenerator(ctx, SortFields, null,
-                ComparatorFactories, RecordDesc, Algorithm.MERGE_SORT, numFramesPerRun);
+        ExternalSortRunGenerator runGenerator = new ExternalSortRunGenerator(ctx, SortFields, null, ComparatorFactories,
+                RecordDesc, Algorithm.MERGE_SORT, numFramesPerRun);
 
         runGenerator.open();
         Map<Integer, String> keyValuePair = new HashMap<>();
@@ -336,20 +335,19 @@
         }
         runGenerator.close();
         List<IFrame> inFrame = new ArrayList<>(runGenerator.getRuns().size());
-        for (RunAndMaxFrameSizePair max : runGenerator.getRuns()) {
-            inFrame.add(new GroupVSizeFrame(ctx, max.maxFrameSize));
+        for (GeneratedRunFileReader max : runGenerator.getRuns()) {
+            inFrame.add(new GroupVSizeFrame(ctx, max.getMaxFrameSize()));
         }
 
         // Let each run file reader not delete the run file when it is read and closed.
-        for (RunAndMaxFrameSizePair run : runGenerator.getRuns()) {
-            RunFileReader runFileReader = (RunFileReader) run.run;
-            PA.setValue(runFileReader, "deleteAfterClose", false);
+        for (GeneratedRunFileReader run : runGenerator.getRuns()) {
+            PA.setValue(run, "deleteAfterClose", false);
         }
         matchResult(ctx, runGenerator.getRuns(), keyValuePair);
 
         List<IFrameReader> runs = new ArrayList<>();
-        for (RunAndMaxFrameSizePair run : runGenerator.getRuns()) {
-            runs.add(run.run);
+        for (GeneratedRunFileReader run : runGenerator.getRuns()) {
+            runs.add(run);
         }
         RunMergingFrameReader reader = new RunMergingFrameReader(ctx, runs, inFrame, SortFields, Comparators, null,
                 RecordDesc);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/SortGroupbyTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/SortGroupbyTest.java
new file mode 100644
index 0000000..bcf661f
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/SortGroupbyTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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 org.apache.hyracks.tests.unit;
+
+import java.util.List;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.io.GeneratedRunFileReader;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.group.sort.ExternalSortGroupByRunGenerator;
+import org.apache.hyracks.dataflow.std.group.sort.ExternalSortGroupByRunMerger;
+import org.apache.hyracks.dataflow.std.sort.AbstractExternalSortRunMerger;
+import org.apache.hyracks.dataflow.std.sort.Algorithm;
+import org.apache.hyracks.dataflow.std.sort.ISorter;
+
+public class SortGroupbyTest extends AbstractExternalGroupbyTest {
+    ExternalSortGroupByRunGenerator builder;
+
+    IOperatorNodePushable mergerOperator;
+
+    @Override
+    protected void initial(final IHyracksTaskContext ctx, int tableSize, final int numFrames)
+            throws HyracksDataException {
+        builder = new ExternalSortGroupByRunGenerator(ctx, keyFields, inRecordDesc, numFrames, keyFields,
+                normalizedKeyComputerFactory, comparatorFactories, partialAggrInState, outputRec, Algorithm.QUICK_SORT);
+
+        mergerOperator = new AbstractUnaryOutputSourceOperatorNodePushable() {
+            @Override
+            public void initialize() throws HyracksDataException {
+                List<GeneratedRunFileReader> runs = builder.getRuns();
+                ISorter sorter = builder.getSorter();
+                IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+                for (int i = 0; i < comparatorFactories.length; ++i) {
+                    comparators[i] = comparatorFactories[i].createBinaryComparator();
+                }
+                INormalizedKeyComputer nmkComputer = normalizedKeyComputerFactory == null ? null
+                        : normalizedKeyComputerFactory.createNormalizedKeyComputer();
+                AbstractExternalSortRunMerger merger = new ExternalSortGroupByRunMerger(ctx, sorter, runs, keyFields,
+                        inRecordDesc, outputRec, outputRec, numFrames, writer, keyFields, nmkComputer, comparators,
+                        partialAggrInState, finalAggrInState, true);
+                merger.process();
+            }
+        };
+    }
+
+    @Override
+    protected IFrameWriter getBuilder() {
+        return builder;
+    }
+
+    @Override
+    protected IOperatorNodePushable getMerger() {
+        return mergerOperator;
+    }
+}
diff --git a/hyracks/hyracks-examples/text-example/textclient/pom.xml b/hyracks/hyracks-examples/text-example/textclient/pom.xml
index 3bf69e2..a88c421 100644
--- a/hyracks/hyracks-examples/text-example/textclient/pom.xml
+++ b/hyracks/hyracks-examples/text-example/textclient/pom.xml
@@ -67,24 +67,7 @@
               <goal>assemble</goal>
             </goals>
           </execution>
-          <execution>
-          	<id>groupclient</id>
-            <configuration>
-              <programs>
-                <program>
-                  <mainClass>org.apache.hyracks.examples.text.client.ExternalGroupClient</mainClass>
-                  <name>groupclient</name>
-                </program>
-              </programs>
-              <repositoryLayout>flat</repositoryLayout>
-              <repositoryName>lib</repositoryName>
-            </configuration>
-            <phase>package</phase>
-            <goals>
-              <goal>assemble</goal>
-            </goals>
-          </execution>
-        </executions>
+       </executions>
       </plugin>
       <plugin>
         <artifactId>maven-assembly-plugin</artifactId>
diff --git a/hyracks/hyracks-examples/text-example/textclient/src/main/java/org/apache/hyracks/examples/text/client/ExternalGroupClient.java b/hyracks/hyracks-examples/text-example/textclient/src/main/java/org/apache/hyracks/examples/text/client/ExternalGroupClient.java
deleted file mode 100644
index 1ae32589..0000000
--- a/hyracks/hyracks-examples/text-example/textclient/src/main/java/org/apache/hyracks/examples/text/client/ExternalGroupClient.java
+++ /dev/null
@@ -1,325 +0,0 @@
-/*
- * 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 org.apache.hyracks.examples.text.client;
-
-import java.io.File;
-
-import org.kohsuke.args4j.CmdLineParser;
-import org.kohsuke.args4j.Option;
-
-import org.apache.hyracks.api.client.HyracksConnection;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
-import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
-import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
-import org.apache.hyracks.api.io.FileReference;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.job.JobSpecification;
-import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
-import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.data.normalizers.IntegerNormalizedKeyComputerFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import org.apache.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.file.FileSplit;
-import org.apache.hyracks.dataflow.std.file.FrameFileWriterOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
-import org.apache.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.group.HashSpillableTableFactory;
-import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
-import org.apache.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
-import org.apache.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
-import org.apache.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
-import org.apache.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-
-/**
- * The application client for the performance tests of the external hash group
- * operator.
- */
-public class ExternalGroupClient {
-    private static class Options {
-        @Option(name = "-host", usage = "Hyracks Cluster Controller Host name", required = true)
-        public String host;
-
-        @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
-        public int port = 1098;
-
-        @Option(name = "-infile-splits", usage = "Comma separated list of file-splits for the input. A file-split is <node-name>:<path>", required = true)
-        public String inFileSplits;
-
-        @Option(name = "-outfile-splits", usage = "Comma separated list of file-splits for the output", required = true)
-        public String outFileSplits;
-
-        @Option(name = "-hashtable-size", usage = "Hash table size (default: 8191)", required = false)
-        public int htSize = 8191;
-
-        @Option(name = "-frame-size", usage = "Frame size (default: 32768)", required = false)
-        public int frameSize = 32768;
-
-        @Option(name = "-sortbuffer-size", usage = "Sort buffer size in frames (default: 512)", required = false)
-        public int sbSize = 512;
-
-        @Option(name = "-sort-output", usage = "Whether to sort the output (default: true)", required = false)
-        public boolean sortOutput = false;
-
-        @Option(name = "-out-plain", usage = "Whether to output plain text (default: true)", required = false)
-        public boolean outPlain = true;
-
-        @Option(name = "-algo", usage = "The algorithm to be used", required = true)
-        public int algo;
-    }
-
-    /**
-     * @param args
-     */
-    public static void main(String[] args) throws Exception {
-        Options options = new Options();
-        CmdLineParser parser = new CmdLineParser(options);
-        parser.parseArgument(args);
-
-        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
-
-        JobSpecification job;
-
-        for (int i = 0; i < 6; i++) {
-            long start = System.currentTimeMillis();
-            job = createJob(parseFileSplits(options.inFileSplits), parseFileSplits(options.outFileSplits, i),
-                    options.htSize, options.sbSize, options.frameSize, options.sortOutput, options.algo,
-                    options.outPlain);
-
-            System.out.print(i + "\t" + (System.currentTimeMillis() - start));
-            start = System.currentTimeMillis();
-            JobId jobId = hcc.startJob(job);
-            hcc.waitForCompletion(jobId);
-            System.out.println("\t" + (System.currentTimeMillis() - start));
-        }
-    }
-
-    private static FileSplit[] parseFileSplits(String fileSplits) {
-        String[] splits = fileSplits.split(",");
-        FileSplit[] fSplits = new FileSplit[splits.length];
-        for (int i = 0; i < splits.length; ++i) {
-            String s = splits[i].trim();
-            int idx = s.indexOf(':');
-            if (idx < 0) {
-                throw new IllegalArgumentException("File split " + s + " not well formed");
-            }
-            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
-        }
-        return fSplits;
-    }
-
-    private static FileSplit[] parseFileSplits(String fileSplits, int count) {
-        String[] splits = fileSplits.split(",");
-        FileSplit[] fSplits = new FileSplit[splits.length];
-        for (int i = 0; i < splits.length; ++i) {
-            String s = splits[i].trim();
-            int idx = s.indexOf(':');
-            if (idx < 0) {
-                throw new IllegalArgumentException("File split " + s + " not well formed");
-            }
-            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1) + "_"
-                    + count)));
-        }
-        return fSplits;
-    }
-
-    private static JobSpecification createJob(FileSplit[] inSplits, FileSplit[] outSplits, int htSize, int sbSize,
-            int frameSize, boolean sortOutput, int alg, boolean outPlain) {
-        JobSpecification spec = new JobSpecification(frameSize);
-        IFileSplitProvider splitsProvider = new ConstantFileSplitProvider(inSplits);
-
-        RecordDescriptor inDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
-                FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
-                new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() });
-
-        FileScanOperatorDescriptor fileScanner = new FileScanOperatorDescriptor(spec, splitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE,
-                        IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
-                        IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
-                        FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, }, '|'), inDesc);
-
-        createPartitionConstraint(spec, fileScanner, inSplits);
-
-        // Output: each unique string with an integer count
-        RecordDescriptor outDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                IntegerSerializerDeserializer.INSTANCE,
-                // IntegerSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE });
-
-        // Specify the grouping key, which will be the string extracted during
-        // the scan.
-        int[] keys = new int[] { 0,
-        // 1
-        };
-
-        AbstractOperatorDescriptor grouper;
-
-        switch (alg) {
-            case 0: // new external hash graph
-                grouper = new org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor(spec,
-                        keys, frameSize, new IBinaryComparatorFactory[] {
-                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                        new IntegerNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
-                                new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(false) }),
-                        new MultiFieldsAggregatorFactory(
-                                new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(keys.length,
-                                        false) }), outDesc, new HashSpillableTableFactory(
-                                new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                                // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }), htSize), false);
-
-                createPartitionConstraint(spec, grouper, outSplits);
-
-                // Connect scanner with the grouper
-                IConnectorDescriptor scanGroupConnDef2 = new MToNPartitioningConnectorDescriptor(spec,
-                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
-                spec.connect(scanGroupConnDef2, fileScanner, 0, grouper, 0);
-
-                break;
-            case 1: // External-sort + new-precluster
-                ExternalSortOperatorDescriptor sorter2 = new ExternalSortOperatorDescriptor(spec, frameSize, keys,
-                        new IBinaryComparatorFactory[] {
-                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, inDesc);
-                createPartitionConstraint(spec, sorter2, inSplits);
-
-                // Connect scan operator with the sorter
-                IConnectorDescriptor scanSortConn2 = new MToNPartitioningConnectorDescriptor(spec,
-                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
-                spec.connect(scanSortConn2, fileScanner, 0, sorter2, 0);
-
-                grouper = new org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor(
-                        spec, keys, new IBinaryComparatorFactory[] {
-                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                        new MultiFieldsAggregatorFactory(
-                                new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                        outDesc);
-
-                createPartitionConstraint(spec, grouper, outSplits);
-
-                // Connect sorter with the pre-cluster
-                OneToOneConnectorDescriptor sortGroupConn2 = new OneToOneConnectorDescriptor(spec);
-                spec.connect(sortGroupConn2, sorter2, 0, grouper, 0);
-                break;
-            case 2: // Inmem
-                grouper = new HashGroupOperatorDescriptor(spec, keys, new FieldHashPartitionComputerFactory(keys,
-                        new IBinaryHashFunctionFactory[] {
-                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }),
-                        new IBinaryComparatorFactory[] {
-                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                        new MultiFieldsAggregatorFactory(
-                                new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                        outDesc, htSize);
-
-                createPartitionConstraint(spec, grouper, outSplits);
-
-                // Connect scanner with the grouper
-                IConnectorDescriptor scanConn2 = new MToNPartitioningConnectorDescriptor(spec,
-                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
-                spec.connect(scanConn2, fileScanner, 0, grouper, 0);
-                break;
-            default:
-                grouper = new org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor(spec,
-                        keys, frameSize, new IBinaryComparatorFactory[] {
-                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                        new IntegerNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
-                                new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(false) }),
-                        new MultiFieldsAggregatorFactory(
-                                new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(keys.length,
-                                        false) }), outDesc, new HashSpillableTableFactory(
-                                new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                                // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
-                                PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }), htSize), false);
-
-                createPartitionConstraint(spec, grouper, outSplits);
-
-                // Connect scanner with the grouper
-                IConnectorDescriptor scanGroupConnDef = new MToNPartitioningConnectorDescriptor(spec,
-                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
-                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
-                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
-                spec.connect(scanGroupConnDef, fileScanner, 0, grouper, 0);
-        }
-
-        IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(outSplits);
-
-        AbstractSingleActivityOperatorDescriptor writer;
-
-        if (outPlain)
-            writer = new PlainFileWriterOperatorDescriptor(spec, outSplitProvider, "|");
-        else
-            writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
-
-        createPartitionConstraint(spec, writer, outSplits);
-
-        IConnectorDescriptor groupOutConn = new OneToOneConnectorDescriptor(spec);
-        spec.connect(groupOutConn, grouper, 0, writer, 0);
-
-        spec.addRoot(writer);
-        return spec;
-    }
-
-    private static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
-        String[] parts = new String[splits.length];
-        for (int i = 0; i < splits.length; ++i) {
-            parts[i] = splits[i].getNodeName();
-        }
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
-    }
-}
diff --git a/hyracks/hyracks-examples/text-example/textclient/src/main/java/org/apache/hyracks/examples/text/client/WordCountMain.java b/hyracks/hyracks-examples/text-example/textclient/src/main/java/org/apache/hyracks/examples/text/client/WordCountMain.java
index b9cbb9d..22a571f 100644
--- a/hyracks/hyracks-examples/text-example/textclient/src/main/java/org/apache/hyracks/examples/text/client/WordCountMain.java
+++ b/hyracks/hyracks-examples/text-example/textclient/src/main/java/org/apache/hyracks/examples/text/client/WordCountMain.java
@@ -21,9 +21,6 @@
 import java.io.File;
 import java.util.EnumSet;
 
-import org.kohsuke.args4j.CmdLineParser;
-import org.kohsuke.args4j.Option;
-
 import org.apache.hyracks.api.client.HyracksConnection;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
@@ -31,6 +28,7 @@
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.io.FileReference;
@@ -39,6 +37,7 @@
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import org.apache.hyracks.data.std.accessors.UTF8StringBinaryHashFunctionFamily;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -52,14 +51,19 @@
 import org.apache.hyracks.dataflow.std.file.FrameFileWriterOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
 import org.apache.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.group.HashSpillableTableFactory;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 import org.apache.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.FloatSumFieldAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
 import org.apache.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
-import org.apache.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
 import org.apache.hyracks.examples.text.WordTupleParserFactory;
+import org.kohsuke.args4j.CmdLineParser;
+import org.kohsuke.args4j.Option;
 
 public class WordCountMain {
     private static class Options {
@@ -84,8 +88,8 @@
         @Option(name = "-hashtable-size", usage = "Hash table size (default: 8191)", required = false)
         public int htSize = 8191;
 
-        @Option(name = "-sortbuffer-size", usage = "Sort buffer size in frames (default: 32768)", required = false)
-        public int sbSize = 32768;
+        @Option(name = "-frame-limit", usage = "Memory limit in frames (default:4)", required = false)
+        public int memFrameLimit = 10;
 
         @Option(name = "-runtime-profiling", usage = "Indicates if runtime profiling should be enabled. (default: false)")
         public boolean runtimeProfiling = false;
@@ -94,6 +98,8 @@
         public int frameSize = 32768;
     }
 
+    private static long fileSize = 0;
+
     public static void main(String[] args) throws Exception {
         Options options = new Options();
         CmdLineParser parser = new CmdLineParser(options);
@@ -102,7 +108,7 @@
         IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
 
         JobSpecification job = createJob(parseFileSplits(options.inFileSplits), parseFileSplits(options.outFileSplits),
-                options.algo, options.htSize, options.sbSize, options.format, options.frameSize);
+                options.algo, options.htSize, options.memFrameLimit, options.format, options.frameSize);
 
         long start = System.currentTimeMillis();
         JobId jobId = hcc.startJob(job,
@@ -121,13 +127,15 @@
             if (idx < 0) {
                 throw new IllegalArgumentException("File split " + s + " not well formed");
             }
-            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
+            File file = new File(s.substring(idx + 1));
+            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(file));
+            fileSize += file.length();
         }
         return fSplits;
     }
 
     private static JobSpecification createJob(FileSplit[] inSplits, FileSplit[] outSplits, String algo, int htSize,
-            int sbSize, String format, int frameSize) {
+            int frameLimit, String format, int frameSize) {
         JobSpecification spec = new JobSpecification(frameSize);
 
         IFileSplitProvider splitsProvider = new ConstantFileSplitProvider(inSplits);
@@ -144,40 +152,39 @@
         IOperatorDescriptor gBy;
         int[] keys = new int[] { 0 };
         if ("hash".equalsIgnoreCase(algo)) {
-            gBy = new HashGroupOperatorDescriptor(
-                    spec,
-                    keys,
-                    new FieldHashPartitionComputerFactory(keys,
-                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                    .of(UTF8StringPointable.FACTORY) }),
+            gBy = new ExternalGroupOperatorDescriptor(spec, htSize, fileSize, keys, frameLimit,
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    new MultiFieldsAggregatorFactory(
-                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                    groupResultDesc, htSize);
+                    new UTF8StringNormalizedKeyComputerFactory(),
+                    new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                            new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(3, false),
+                            new FloatSumFieldAggregatorFactory(5, false) }),
+                    new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                            new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(2, false),
+                            new FloatSumFieldAggregatorFactory(3, false) }),
+                    groupResultDesc, groupResultDesc, new HashSpillableTableFactory(
+                            new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE }));
+
             createPartitionConstraint(spec, gBy, outSplits);
             IConnectorDescriptor scanGroupConn = new MToNPartitioningConnectorDescriptor(spec,
-                    new FieldHashPartitionComputerFactory(keys,
-                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                    .of(UTF8StringPointable.FACTORY) }));
+                    new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+                            PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
             spec.connect(scanGroupConn, wordScanner, 0, gBy, 0);
         } else {
-            IBinaryComparatorFactory[] cfs = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
-                    .of(UTF8StringPointable.FACTORY) };
-            IOperatorDescriptor sorter = "memsort".equalsIgnoreCase(algo) ? new InMemorySortOperatorDescriptor(spec,
-                    keys, new UTF8StringNormalizedKeyComputerFactory(), cfs, wordDesc)
-                    : new ExternalSortOperatorDescriptor(spec, sbSize, keys,
+            IBinaryComparatorFactory[] cfs = new IBinaryComparatorFactory[] {
+                    PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
+            IOperatorDescriptor sorter = "memsort".equalsIgnoreCase(algo)
+                    ? new InMemorySortOperatorDescriptor(spec, keys, new UTF8StringNormalizedKeyComputerFactory(), cfs,
+                            wordDesc)
+                    : new ExternalSortOperatorDescriptor(spec, frameLimit, keys,
                             new UTF8StringNormalizedKeyComputerFactory(), cfs, wordDesc);
             createPartitionConstraint(spec, sorter, outSplits);
 
             IConnectorDescriptor scanSortConn = new MToNPartitioningConnectorDescriptor(spec,
-                    new FieldHashPartitionComputerFactory(keys,
-                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                    .of(UTF8StringPointable.FACTORY) }));
+                    new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+                            PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
             spec.connect(scanSortConn, wordScanner, 0, sorter, 0);
 
-            gBy = new PreclusteredGroupOperatorDescriptor(
-                    spec,
-                    keys,
+            gBy = new PreclusteredGroupOperatorDescriptor(spec, keys,
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                     new MultiFieldsAggregatorFactory(
                             new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
@@ -188,8 +195,9 @@
         }
 
         IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(outSplits);
-        IOperatorDescriptor writer = "text".equalsIgnoreCase(format) ? new PlainFileWriterOperatorDescriptor(spec,
-                outSplitProvider, ",") : new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
+        IOperatorDescriptor writer = "text".equalsIgnoreCase(format)
+                ? new PlainFileWriterOperatorDescriptor(spec, outSplitProvider, ",")
+                : new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
         createPartitionConstraint(spec, writer, outSplits);
 
         IConnectorDescriptor gbyPrinterConn = new OneToOneConnectorDescriptor(spec);
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
index cd45536..a52d21e 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -27,44 +27,79 @@
     <version>0.2.17-SNAPSHOT</version>
   </parent>
 
-  <dependencies>
-  	<dependency>
-  		<groupId>org.apache.hyracks</groupId>
-  		<artifactId>hyracks-dataflow-std</artifactId>
-  		<version>0.2.17-SNAPSHOT</version>
-  		<scope>compile</scope>
-  	</dependency>
-  	<dependency>
-  		<groupId>org.apache.hyracks</groupId>
-  		<artifactId>hyracks-data-std</artifactId>
-  		<version>0.2.17-SNAPSHOT</version>
-  	</dependency>
-  </dependencies>
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>appassembler-maven-plugin</artifactId>
-        <version>1.3</version>
-        <executions>
-          <execution>
-            <configuration>
-              <programs>
-                <program>
-                  <mainClass>org.apache.hyracks.examples.tpch.client.Sort</mainClass>
-                  <name>tpchclient</name>
-                </program>
-              </programs>
-              <repositoryLayout>flat</repositoryLayout>
-              <repositoryName>lib</repositoryName>
-            </configuration>
-            <phase>package</phase>
-            <goals>
-              <goal>assemble</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-dataflow-std</artifactId>
+            <version>0.2.17-SNAPSHOT</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hyracks</groupId>
+            <artifactId>hyracks-data-std</artifactId>
+            <version>0.2.17-SNAPSHOT</version>
+        </dependency>
+    </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>appassembler-maven-plugin</artifactId>
+                <version>1.3</version>
+                <executions>
+                    <execution>
+                        <id>sort</id>
+                        <configuration>
+                            <programs>
+                                <program>
+                                    <mainClass>org.apache.hyracks.examples.tpch.client.Sort</mainClass>
+                                    <name>sort</name>
+                                </program>
+                            </programs>
+                            <repositoryLayout>flat</repositoryLayout>
+                            <repositoryName>lib</repositoryName>
+                        </configuration>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>assemble</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>group</id>
+                        <configuration>
+                            <programs>
+                                <program>
+                                    <mainClass>org.apache.hyracks.examples.tpch.client.Groupby</mainClass>
+                                    <name>group</name>
+                                </program>
+                            </programs>
+                            <repositoryLayout>flat</repositoryLayout>
+                            <repositoryName>lib</repositoryName>
+                        </configuration>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>assemble</goal>
+                        </goals>
+                    </execution>
+                    <execution>
+                        <id>join</id>
+                        <configuration>
+                            <programs>
+                                <program>
+                                    <mainClass>org.apache.hyracks.examples.tpch.client.Join</mainClass>
+                                    <name>join</name>
+                                </program>
+                            </programs>
+                            <repositoryLayout>flat</repositoryLayout>
+                            <repositoryName>lib</repositoryName>
+                        </configuration>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>assemble</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 </project>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Common.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Common.java
index ac172fd..15d7f66 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Common.java
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Common.java
@@ -27,8 +27,12 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.io.FileReference;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import org.apache.hyracks.dataflow.std.file.FileSplit;
 
@@ -56,12 +60,36 @@
             new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
             new UTF8StringSerializerDeserializer() });
 
-    static IValueParserFactory[] orderParserFactories = new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE };
+    static RecordDescriptor lineitemDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+            FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(),
+            new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() });
 
+    static IValueParserFactory[] lineitemParserFactories = new IValueParserFactory[] {
+            IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+            IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+            IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
+            FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, };
+
+    static IValueParserFactory[] custParserFactories = new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE };
+    static IValueParserFactory[] orderParserFactories = new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+            UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE };
 
     static FileSplit[] parseFileSplits(String fileSplits) {
         String[] splits = fileSplits.split(",");
@@ -77,6 +105,21 @@
         return fSplits;
     }
 
+    static FileSplit[] parseFileSplits(String fileSplits, int count) {
+        String[] splits = fileSplits.split(",");
+        FileSplit[] fSplits = new FileSplit[splits.length];
+        for (int i = 0; i < splits.length; ++i) {
+            String s = splits[i].trim();
+            int idx = s.indexOf(':');
+            if (idx < 0) {
+                throw new IllegalArgumentException("File split " + s + " not well formed");
+            }
+            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1) + "_"
+                    + count)));
+        }
+        return fSplits;
+    }
+
     static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
         String[] parts = new String[splits.length];
         for (int i = 0; i < splits.length; ++i) {
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Groupby.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Groupby.java
new file mode 100644
index 0000000..2ef5835
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Groupby.java
@@ -0,0 +1,207 @@
+/*
+ * 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 org.apache.hyracks.examples.tpch.client;
+
+import static org.apache.hyracks.examples.tpch.client.Common.createPartitionConstraint;
+import static org.apache.hyracks.examples.tpch.client.Common.lineitemDesc;
+import static org.apache.hyracks.examples.tpch.client.Common.lineitemParserFactories;
+import static org.apache.hyracks.examples.tpch.client.Common.parseFileSplits;
+
+import org.apache.hyracks.api.client.HyracksConnection;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.data.std.accessors.MurmurHash3BinaryHashFunctionFamily;
+import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.normalizers.IntegerNormalizedKeyComputerFactory;
+import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import org.apache.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.file.FileSplit;
+import org.apache.hyracks.dataflow.std.file.FrameFileWriterOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.group.HashSpillableTableFactory;
+import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.group.sort.SortGroupByOperatorDescriptor;
+import org.kohsuke.args4j.CmdLineParser;
+import org.kohsuke.args4j.Option;
+
+/**
+ * The application client for the performance tests of the groupby
+ * operator.
+ */
+public class Groupby {
+    private static class Options {
+        @Option(name = "-host", usage = "Hyracks Cluster Controller Host name", required = true)
+        public String host;
+
+        @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)")
+        public int port = 1098;
+
+        @Option(name = "-infile-splits", usage = "Comma separated list of file-splits for the input. A file-split is <node-name>:<path>", required = true)
+        public String inFileSplits;
+
+        @Option(name = "-outfile-splits", usage = "Comma separated list of file-splits for the output", required = true)
+        public String outFileSplits;
+
+        @Option(name = "-input-tuples", usage = "Hash table size ", required = true)
+        public int htSize;
+
+        @Option(name = "-input-size", usage = "Physical file size in bytes ", required = true)
+        public long fileSize;
+
+        @Option(name = "-frame-size", usage = "Frame size (default: 32768)", required = false)
+        public int frameSize = 32768;
+
+        @Option(name = "-frame-limit", usage = "memory limit for sorting (default: 4)", required = false)
+        public int frameLimit = 4;
+
+        @Option(name = "-out-plain", usage = "Whether to output plain text (default: true)", required = false)
+        public boolean outPlain = true;
+
+        @Option(name = "-algo", usage = "The algorithm to be used: hash|sort", required = true)
+        public String algo = "hash";
+    }
+
+    public static void main(String[] args) throws Exception {
+        Options options = new Options();
+        CmdLineParser parser = new CmdLineParser(options);
+        if (args.length == 0) {
+            parser.printUsage(System.err);
+            return;
+        }
+        parser.parseArgument(args);
+
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
+
+        JobSpecification job;
+
+        long start = System.currentTimeMillis();
+        job = createJob(parseFileSplits(options.inFileSplits), parseFileSplits(options.outFileSplits), options.htSize,
+                options.fileSize, options.frameLimit, options.frameSize, options.algo, options.outPlain);
+        if (job != null) {
+            System.out.print("CreateJobTime:" + (System.currentTimeMillis() - start));
+            start = System.currentTimeMillis();
+            JobId jobId = hcc.startJob(job);
+            hcc.waitForCompletion(jobId);
+            System.out.println("JobExecuteTime:" + (System.currentTimeMillis() - start));
+        }
+    }
+
+    private static JobSpecification createJob(FileSplit[] inSplits, FileSplit[] outSplits, int htSize, long fileSize,
+            int frameLimit, int frameSize, String alg, boolean outPlain) {
+        JobSpecification spec = new JobSpecification(frameSize);
+        IFileSplitProvider splitsProvider = new ConstantFileSplitProvider(inSplits);
+
+        FileScanOperatorDescriptor fileScanner = new FileScanOperatorDescriptor(spec, splitsProvider,
+                new DelimitedDataTupleParserFactory(lineitemParserFactories, '|'), lineitemDesc);
+
+        createPartitionConstraint(spec, fileScanner, inSplits);
+
+        // Output: each unique string with an integer count
+        RecordDescriptor outDesc = new RecordDescriptor(
+                new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE,
+                        // IntegerSerializerDeserializer.INSTANCE,
+                        IntegerSerializerDeserializer.INSTANCE });
+
+        // Specify the grouping key, which will be the string extracted during
+        // the scan.
+        int[] keys = new int[] { 0,
+                // 1
+        };
+
+        AbstractOperatorDescriptor grouper;
+
+        if (alg.equalsIgnoreCase("hash")) {// external hash graph
+            grouper = new ExternalGroupOperatorDescriptor(spec, htSize, fileSize, keys, frameLimit,
+                    new IBinaryComparatorFactory[] {
+                            // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                            PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                    new IntegerNormalizedKeyComputerFactory(),
+                    new MultiFieldsAggregatorFactory(
+                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(false) }),
+                    new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                            new IntSumFieldAggregatorFactory(keys.length, false) }),
+                    outDesc, outDesc, new HashSpillableTableFactory(
+                            new IBinaryHashFunctionFamily[] { MurmurHash3BinaryHashFunctionFamily.INSTANCE }));
+
+            createPartitionConstraint(spec, grouper, outSplits);
+        } else if (alg.equalsIgnoreCase("sort")) {
+            grouper = new SortGroupByOperatorDescriptor(spec, frameLimit, keys, keys,
+                    new IntegerNormalizedKeyComputerFactory(),
+                    new IBinaryComparatorFactory[] {
+                            // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                            PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+                    new MultiFieldsAggregatorFactory(
+                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
+                    new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                            new IntSumFieldAggregatorFactory(keys.length, true) }),
+                    outDesc, outDesc, false);
+
+            createPartitionConstraint(spec, grouper, outSplits);
+        } else {
+            System.err.println("unknow groupby alg:" + alg);
+            return null;
+        }
+        // Connect scanner with the grouper
+        IConnectorDescriptor scanGroupConnDef2 = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(keys,
+                        new IBinaryHashFunctionFactory[] {
+                                // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+                                PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
+        spec.connect(scanGroupConnDef2, fileScanner, 0, grouper, 0);
+
+        IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(outSplits);
+
+        AbstractSingleActivityOperatorDescriptor writer;
+
+        if (outPlain)
+            writer = new PlainFileWriterOperatorDescriptor(spec, outSplitProvider, "|");
+        else
+            writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
+
+        createPartitionConstraint(spec, writer, outSplits);
+
+        IConnectorDescriptor groupOutConn = new OneToOneConnectorDescriptor(spec);
+        spec.connect(groupOutConn, grouper, 0, writer, 0);
+
+        spec.addRoot(writer);
+        return spec;
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Join.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Join.java
index fbfc5f5..507e1c7 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Join.java
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Join.java
@@ -18,26 +18,22 @@
  */
 package org.apache.hyracks.examples.tpch.client;
 
-import static org.apache.hyracks.examples.tpch.client.Common.*;
+import static org.apache.hyracks.examples.tpch.client.Common.createPartitionConstraint;
+import static org.apache.hyracks.examples.tpch.client.Common.custParserFactories;
+import static org.apache.hyracks.examples.tpch.client.Common.orderParserFactories;
+import static org.apache.hyracks.examples.tpch.client.Common.parseFileSplits;
 
 import java.util.EnumSet;
 
-import org.kohsuke.args4j.CmdLineParser;
-import org.kohsuke.args4j.Option;
-
 import org.apache.hyracks.api.client.HyracksConnection;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.constraints.PartitionConstraintHelper;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.ITuplePairComparator;
-import org.apache.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.JobFlag;
@@ -45,11 +41,11 @@
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import org.apache.hyracks.data.std.accessors.UTF8StringBinaryHashFunctionFamily;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import org.apache.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
 import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
 import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
@@ -57,16 +53,21 @@
 import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
 import org.apache.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.file.FileSplit;
-import org.apache.hyracks.dataflow.std.file.FrameFileWriterOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
+import org.apache.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.group.HashSpillableTableFactory;
 import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
-import org.apache.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.FloatSumFieldAggregatorFactory;
+import org.apache.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
 import org.apache.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
-import org.apache.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.join.GraceHashJoinOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.join.HybridHashJoinOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.join.JoinComparatorFactory;
 import org.apache.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.join.OptimizedHybridHashJoinOperatorDescriptor;
+import org.kohsuke.args4j.CmdLineParser;
+import org.kohsuke.args4j.Option;
 
 public class Join {
     private static class Options {
@@ -91,23 +92,23 @@
         @Option(name = "-profile", usage = "Enable/Disable profiling. (default: enabled)")
         public boolean profile = true;
 
-        @Option(name = "-table-size", usage = "Table size for in-memory hash join", required = false)
+        @Option(name = "-table-size", usage = "Table size for in-memory hash join. (default: 8191)", required = false)
         public int tableSize = 8191;
 
-        @Option(name = "-algo", usage = "Join types", required = true)
+        @Option(name = "-algo", usage = "Join types:InMem|NestedLoop|Hybrid|Grace", required = true)
         public String algo;
 
         // For grace/hybrid hash join only
         @Option(name = "-mem-size", usage = "Memory size for hash join", required = true)
         public int memSize;
 
-        @Option(name = "-input-size", usage = "Input size of the grace/hybrid hash join", required = false)
-        public int graceInputSize = 10;
+        @Option(name = "-input-size", usage = "Input size of the hybrid hash join", required = false)
+        public int graceInputSize = 100000;
 
-        @Option(name = "-records-per-frame", usage = "Records per frame for grace/hybrid hash join", required = false)
+        @Option(name = "-records-per-frame", usage = "Records per frame for hybrid hash join", required = false)
         public int graceRecordsPerFrame = 200;
 
-        @Option(name = "-grace-factor", usage = "Factor of the grace/hybrid hash join", required = false)
+        @Option(name = "-grace-factor", usage = "Factor of the grace/hybrid hash join, (default:1.2)", required = false)
         public double graceFactor = 1.2;
 
         // Whether group-by is processed after the join
@@ -121,6 +122,10 @@
     public static void main(String[] args) throws Exception {
         Options options = new Options();
         CmdLineParser parser = new CmdLineParser(options);
+        if (args.length == 0) {
+            parser.printUsage(System.err);
+            return;
+        }
         parser.parseArgument(args);
 
         IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
@@ -129,6 +134,9 @@
                 parseFileSplits(options.inFileOrderSplits), parseFileSplits(options.outFileSplits),
                 options.numJoinPartitions, options.algo, options.graceInputSize, options.graceRecordsPerFrame,
                 options.graceFactor, options.memSize, options.tableSize, options.hasGroupBy, options.frameSize);
+        if (job == null) {
+            return;
+        }
 
         long start = System.currentTimeMillis();
         JobId jobId = hcc.startJob(job,
@@ -141,87 +149,76 @@
     private static JobSpecification createJob(FileSplit[] customerSplits, FileSplit[] orderSplits,
             FileSplit[] resultSplits, int numJoinPartitions, String algo, int graceInputSize, int graceRecordsPerFrame,
             double graceFactor, int memSize, int tableSize, boolean hasGroupBy, int frameSize)
-            throws HyracksDataException {
+                    throws HyracksDataException {
         JobSpecification spec = new JobSpecification(frameSize);
 
         IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(customerSplits);
+        long custFileSize = 0;
+        for (int i = 0; i < customerSplits.length; i++) {
+            custFileSize += customerSplits[i].getLocalFile().getFile().length();
+        }
 
         IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(orderSplits);
+        long orderFileSize = 0;
+        for (int i = 0; i < orderSplits.length; i++) {
+            orderFileSize += orderSplits[i].getLocalFile().getFile().length();
+        }
 
         FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), Common.ordersDesc);
+                new DelimitedDataTupleParserFactory(orderParserFactories, '|'), Common.ordersDesc);
         createPartitionConstraint(spec, ordScanner, orderSplits);
 
         FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), Common.custDesc);
+                new DelimitedDataTupleParserFactory(custParserFactories, '|'), Common.custDesc);
         createPartitionConstraint(spec, custScanner, customerSplits);
 
         IOperatorDescriptor join;
 
         if ("nestedloop".equalsIgnoreCase(algo)) {
-            join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                    PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), Common.custOrderJoinDesc,
-                    memSize, false, null);
+            join = new NestedLoopJoinOperatorDescriptor(spec,
+                    new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
+                    Common.custOrderJoinDesc, memSize, false, null);
 
-        } else if ("gracehash".equalsIgnoreCase(algo)) {
-            join = new GraceHashJoinOperatorDescriptor(
-                    spec,
-                    memSize,
-                    graceInputSize,
-                    graceRecordsPerFrame,
-                    graceFactor,
-                    new int[] { 0 },
-                    new int[] { 1 },
-                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                            .of(UTF8StringPointable.FACTORY) },
+        } else if ("inmem".equalsIgnoreCase(algo)) {
+            join = new InMemoryHashJoinOperatorDescriptor(spec, new int[] { 0 }, new int[] { 1 },
+                    new IBinaryHashFunctionFactory[] {
+                            PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    Common.custOrderJoinDesc, null);
+                    Common.custOrderJoinDesc, tableSize, null);
 
-        } else if ("hybridhash".equalsIgnoreCase(algo)) {
-            join = new HybridHashJoinOperatorDescriptor(
-                    spec,
-                    memSize,
-                    graceInputSize,
-                    graceRecordsPerFrame,
-                    graceFactor,
-                    new int[] { 0 },
-                    new int[] { 1 },
-                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                            .of(UTF8StringPointable.FACTORY) },
+        } else if ("hybrid".equalsIgnoreCase(algo)) {
+            join = new OptimizedHybridHashJoinOperatorDescriptor(spec, memSize, graceInputSize, graceFactor,
+                    new int[] { 0 }, new int[] { 1 },
+                    new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE },
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                    Common.custOrderJoinDesc,
+                    new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1),
+                    new JoinComparatorFactory(PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0),
+                    null);
+
+        } else if ("grace".equalsIgnoreCase(algo)) {
+            join = new GraceHashJoinOperatorDescriptor(spec, memSize, graceInputSize, graceRecordsPerFrame, graceFactor,
+                    new int[] { 0 }, new int[] { 1 },
+                    new IBinaryHashFunctionFactory[] {
+                            PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) },
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
                     Common.custOrderJoinDesc, null);
 
         } else {
-            join = new InMemoryHashJoinOperatorDescriptor(
-                    spec,
-                    new int[] { 0 },
-                    new int[] { 1 },
-                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                            .of(UTF8StringPointable.FACTORY) },
-                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    Common.custOrderJoinDesc, 6000000, null);
+            System.err.println("unknown algorithm:" + algo);
+            return null;
         }
 
         PartitionConstraintHelper.addPartitionCountConstraint(spec, join, numJoinPartitions);
 
         IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(new int[] { 1 }, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(ordJoinConn, ordScanner, 0, join, 1);
 
         IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
+                new FieldHashPartitionComputerFactory(new int[] { 0 }, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
         spec.connect(custJoinConn, custScanner, 0, join, 0);
 
         IOperatorDescriptor endingOp = join;
@@ -231,29 +228,33 @@
             RecordDescriptor groupResultDesc = new RecordDescriptor(new ISerializerDeserializer[] {
                     new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE });
 
-            HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(
-                    spec,
-                    new int[] { 6 },
-                    new FieldHashPartitionComputerFactory(new int[] { 6 },
-                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                    .of(UTF8StringPointable.FACTORY) }),
+            ExternalGroupOperatorDescriptor gby = new ExternalGroupOperatorDescriptor(spec, tableSize,
+                    custFileSize + orderFileSize, new int[] { 6 }, memSize,
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    new MultiFieldsAggregatorFactory(
-                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                    groupResultDesc, 16);
+                    new UTF8StringNormalizedKeyComputerFactory(),
+                    new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                            new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(3, false),
+                            new FloatSumFieldAggregatorFactory(5, false) }),
+                    new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+                            new IntSumFieldAggregatorFactory(1, false), new IntSumFieldAggregatorFactory(2, false),
+                            new FloatSumFieldAggregatorFactory(3, false) }),
+                    groupResultDesc, groupResultDesc, new HashSpillableTableFactory(
+                            new IBinaryHashFunctionFamily[] { UTF8StringBinaryHashFunctionFamily.INSTANCE }));
+
             createPartitionConstraint(spec, gby, resultSplits);
 
             IConnectorDescriptor joinGroupConn = new MToNPartitioningConnectorDescriptor(spec,
-                    new FieldHashPartitionComputerFactory(new int[] { 6 },
-                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                    .of(UTF8StringPointable.FACTORY) }));
+                    new FieldHashPartitionComputerFactory(new int[] { 6 }, new IBinaryHashFunctionFactory[] {
+                            PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
             spec.connect(joinGroupConn, join, 0, gby, 0);
 
             endingOp = gby;
         }
 
         IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(resultSplits);
-        FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
+        //FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
+        IOperatorDescriptor writer = new PlainFileWriterOperatorDescriptor(spec, outSplitProvider, "|");
+
         createPartitionConstraint(spec, writer, resultSplits);
 
         IConnectorDescriptor endingPrinterConn = new OneToOneConnectorDescriptor(spec);
@@ -262,63 +263,4 @@
         spec.addRoot(writer);
         return spec;
     }
-
-
-
-    static class JoinComparatorFactory implements ITuplePairComparatorFactory {
-        private static final long serialVersionUID = 1L;
-
-        private final IBinaryComparatorFactory bFactory;
-        private final int pos0;
-        private final int pos1;
-
-        public JoinComparatorFactory(IBinaryComparatorFactory bFactory, int pos0, int pos1) {
-            this.bFactory = bFactory;
-            this.pos0 = pos0;
-            this.pos1 = pos1;
-        }
-
-        @Override
-        public ITuplePairComparator createTuplePairComparator(IHyracksTaskContext ctx) {
-            return new JoinComparator(bFactory.createBinaryComparator(), pos0, pos1);
-        }
-    }
-
-    static class JoinComparator implements ITuplePairComparator {
-
-        private final IBinaryComparator bComparator;
-        private final int field0;
-        private final int field1;
-
-        public JoinComparator(IBinaryComparator bComparator, int field0, int field1) {
-            this.bComparator = bComparator;
-            this.field0 = field0;
-            this.field1 = field1;
-        }
-
-        @Override
-        public int compare(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
-                throws HyracksDataException {
-            int tStart0 = accessor0.getTupleStartOffset(tIndex0);
-            int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
-
-            int tStart1 = accessor1.getTupleStartOffset(tIndex1);
-            int fStartOffset1 = accessor1.getFieldSlotsLength() + tStart1;
-
-            int fStart0 = accessor0.getFieldStartOffset(tIndex0, field0);
-            int fEnd0 = accessor0.getFieldEndOffset(tIndex0, field0);
-            int fLen0 = fEnd0 - fStart0;
-
-            int fStart1 = accessor1.getFieldStartOffset(tIndex1, field1);
-            int fEnd1 = accessor1.getFieldEndOffset(tIndex1, field1);
-            int fLen1 = fEnd1 - fStart1;
-
-            int c = bComparator.compare(accessor0.getBuffer().array(), fStart0 + fStartOffset0, fLen0, accessor1
-                    .getBuffer().array(), fStart1 + fStartOffset1, fLen1);
-            if (c != 0) {
-                return c;
-            }
-            return 0;
-        }
-    }
 }
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Sort.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Sort.java
index f2dd519..28e6dd9 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Sort.java
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/org/apache/hyracks/examples/tpch/client/Sort.java
@@ -42,6 +42,7 @@
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
 import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy;
 import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
@@ -54,7 +55,6 @@
 import org.apache.hyracks.dataflow.std.sort.Algorithm;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import org.apache.hyracks.dataflow.std.sort.TopKSorterOperatorDescriptor;
-import org.apache.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
 
 public class Sort {
     private static class Options {
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
index 57e524a..5a7a4a7 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
@@ -1,8 +1,3 @@
-package org.apache.hyracks.storage.am.common.api;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -21,6 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
+package org.apache.hyracks.storage.am.common.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
 public interface ITwoPCIndexBulkLoader {
 
     /**
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index 1b1bee0..170950e 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -81,6 +81,7 @@
 
                     FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
                             tb.getSize());
+
                 }
                 appender.write(writer, true);
             } catch (Throwable th) {
diff --git a/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java b/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
index b677ee1..f796812 100644
--- a/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java
@@ -19,6 +19,8 @@
 package org.apache.hyracks.test.support;
 
 import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.ExecutorService;
 
 import org.apache.hyracks.api.context.IHyracksJobletContext;
@@ -40,6 +42,8 @@
     private final TaskAttemptId taskId;
     private WorkspaceFileFactory fileFactory;
 
+    private Map<Object, IStateObject> stateObjectMap = new HashMap<>();
+
     public TestTaskContext(TestJobletContext jobletContext, TaskAttemptId taskId) {
         this.jobletContext = jobletContext;
         this.taskId = taskId;
@@ -114,13 +118,13 @@
     }
 
     @Override
-    public void setStateObject(IStateObject taskState) {
-
+    public synchronized void setStateObject(IStateObject taskState) {
+        stateObjectMap.put(taskState.getId(), taskState);
     }
 
     @Override
-    public IStateObject getStateObject(Object id) {
-        return null;
+    public synchronized IStateObject getStateObject(Object id) {
+        return stateObjectMap.get(id);
     }
 
     @Override
diff --git a/pom.xml b/pom.xml
index 90f80a1..fe44a12 100644
--- a/pom.xml
+++ b/pom.xml
@@ -171,6 +171,7 @@
            <exclude>**/ClusterControllerService/**</exclude>
            <exclude>**/target/**</exclude>
            <exclude>**/output/**</exclude>
+           <exclude>**/target/**</exclude>
            <exclude>**/*.iml</exclude>
            <exclude>**/*.prefs</exclude>
            <exclude>**/.classpath</exclude>