merge hyracks_dev_next r978:1064

git-svn-id: https://hyracks.googlecode.com/svn/branches/aggregators_dev_next@1065 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
index c0c369e..f13b2bb 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
@@ -46,9 +46,9 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableGroupingTableFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableTableFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
 public class ExternalGroupByPOperator extends AbstractPhysicalOperator {
 
@@ -235,7 +235,7 @@
                 gbyCols, aggOpInputEnv, context);
         ExternalGroupOperatorDescriptor gbyOpDesc = new ExternalGroupOperatorDescriptor(spec, keyAndDecFields,
                 frameLimit, comparatorFactories, normalizedKeyFactory, aggregatorFactory, mergeFactory,
-                recordDescriptor, new HashSpillableGroupingTableFactory(tpcf, tableSize), false);
+                recordDescriptor, new HashSpillableTableFactory(tpcf, tableSize), false);
 
         contributeOpDesc(builder, gby, gbyOpDesc);
         ILogicalOperator src = op.getInputs().get(0).getValue();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
index bf59352..5b02d83 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
@@ -23,7 +23,7 @@
 import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
 public class MicroPreclusteredGroupByPOperator extends AbstractPreclusteredGroupByPOperator {
 
@@ -62,7 +62,7 @@
         }
         // compile subplans and set the gby op. schema accordingly
         AlgebricksPipeline[] subplans = compileSubplans(inputSchemas[0], gby, opSchema, context);
-        IAccumulatingAggregatorFactory aggregatorFactory = new NestedPlansAccumulatingAggregatorFactory(subplans, keys,
+        IAggregatorDescriptorFactory aggregatorFactory = new NestedPlansAccumulatingAggregatorFactory(subplans, keys,
                 fdColumns);
 
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
index c97e57f..d74fb62 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
@@ -44,7 +44,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
 
 public class PreSortedDistinctByPOperator extends AbstractPhysicalOperator {
@@ -108,7 +108,7 @@
         IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
                 columnList, context.getTypeEnvironment(op), context);
         IAggregateFunctionFactory[] aggFactories = new IAggregateFunctionFactory[] {};
-        IAccumulatingAggregatorFactory aggregatorFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(
+        IAggregatorDescriptorFactory aggregatorFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(
                 aggFactories, keys, fdColumns);
 
         RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
index 780e4b4..bb19bb9 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
@@ -37,7 +37,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
 
 public class PreclusteredGroupByPOperator extends AbstractPreclusteredGroupByPOperator {
@@ -76,7 +76,7 @@
         }
         // compile subplans and set the gby op. schema accordingly
         AlgebricksPipeline[] subplans = compileSubplans(inputSchemas[0], gby, opSchema, context);
-        IAccumulatingAggregatorFactory aggregatorFactory = new NestedPlansAccumulatingAggregatorFactory(subplans, keys,
+        IAggregatorDescriptorFactory aggregatorFactory = new NestedPlansAccumulatingAggregatorFactory(subplans, keys,
                 fdColumns);
 
         JobSpecification spec = builder.getJobSpec();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
index 4b232a8..b7b64ab 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
@@ -29,11 +29,11 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregator;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
-public class NestedPlansAccumulatingAggregatorFactory implements IAccumulatingAggregatorFactory {
+public class NestedPlansAccumulatingAggregatorFactory implements IAggregatorDescriptorFactory {
 
     private static final long serialVersionUID = 1L;
     private AlgebricksPipeline[] subplans;
@@ -48,8 +48,8 @@
     }
 
     @Override
-    public IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
-            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
+            RecordDescriptor outRecordDescriptor, int[] keys, int[] partialKeys) throws HyracksDataException {
 
         final RuntimeContext rc = new RuntimeContext();
         rc.setHyracksContext(ctx);
@@ -64,12 +64,11 @@
             }
         }
 
-        return new IAccumulatingAggregator() {
-
-            private boolean pending;
+        return new IAggregatorDescriptor() {
 
             @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+            public void init(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
                 ArrayTupleBuilder tb = outputWriter.getTupleBuilder();
                 tb.reset();
                 for (int i = 0; i < keyFieldIdx.length; ++i) {
@@ -81,11 +80,16 @@
                 for (int i = 0; i < pipelines.length; ++i) {
                     pipelines[i].open();
                 }
-                pending = false;
+
+                // aggregate the first tuple
+                for (int i = 0; i < pipelines.length; i++) {
+                    pipelines[i].writeTuple(accessor.getBuffer(), tIndex);
+                }
             }
 
             @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+            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++) {
@@ -94,22 +98,48 @@
             }
 
             @Override
-            public boolean output(FrameTupleAppender appender, IFrameTupleAccessor accessor, int tIndex,
-                    int[] keyFieldIndexes) throws HyracksDataException {
-                if (!pending) {
-                    for (int i = 0; i < pipelines.length; i++) {
-                        outputWriter.setInputIdx(i);
-                        pipelines[i].close();
-                    }
+            public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                for (int i = 0; i < pipelines.length; i++) {
+                    outputWriter.setInputIdx(i);
+                    pipelines[i].close();
                 }
-                if (!outputWriter.writeTuple(appender)) {
-                    pending = true;
-                    return false;
-                } else {
-                    return true;
+                // outputWriter.writeTuple(appender);
+                tupleBuilder.reset();
+                ArrayTupleBuilder tb = outputWriter.getTupleBuilder();
+                byte[] data = tb.getByteArray();
+                int[] fieldEnds = tb.getFieldEndOffsets();
+                int start = 0;
+                int offset = 0;
+                for (int i = 0; i < fieldEnds.length; i++) {
+                    if (i > 0)
+                        start = fieldEnds[i - 1];
+                    offset = fieldEnds[i] - start;
+                    tupleBuilder.addField(data, start, offset);
                 }
             }
 
+            @Override
+            public AggregateState createAggregateStates() {
+                return new AggregateState();
+            }
+
+            @Override
+            public void reset() {
+
+            }
+
+            @Override
+            public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                throw new IllegalStateException("this method should not be called");
+            }
+
+            @Override
+            public void close() {
+
+            }
+
         };
     }
 
@@ -168,6 +198,7 @@
 
         @Override
         public void open() throws HyracksDataException {
+
         }
 
         /**
@@ -196,10 +227,6 @@
             this.inputIdx = inputIdx;
         }
 
-        public boolean writeTuple(FrameTupleAppender appender) {
-            return appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
-        }
-
         public ArrayTupleBuilder getTupleBuilder() {
             return tb;
         }
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
index 9082ec1..abfd1c1 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SerializableAggregatorDescriptorFactory.java
@@ -11,12 +11,12 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
 public class SerializableAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
     private static final long serialVersionUID = 1L;
-
     private ISerializableAggregateFunctionFactory[] aggFactories;
 
     public SerializableAggregatorDescriptorFactory(ISerializableAggregateFunctionFactory[] aggFactories) {
@@ -25,9 +25,9 @@
 
     @Override
     public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, int[] keyFields) throws HyracksDataException {
+            RecordDescriptor outRecordDescriptor, int[] keyFields, final int[] keyFieldsInPartialResults)
+            throws HyracksDataException {
         final int[] keys = keyFields;
-        final int OFFSET_INT_LENGTH = 4;
 
         /**
          * one IAggregatorDescriptor instance per Gby operator
@@ -39,11 +39,15 @@
             private int stateFieldLength[] = new int[aggFactories.length];
 
             @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tb)
+            public AggregateState createAggregateStates() {
+                return null;
+            }
+
+            @Override
+            public void init(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex, AggregateState state)
                     throws HyracksDataException {
                 DataOutput output = tb.getDataOutput();
                 ftr.reset(accessor, tIndex);
-                int startSize = tb.getSize();
                 for (int i = 0; i < aggs.length; i++) {
                     try {
                         int begin = tb.getSize();
@@ -57,35 +61,41 @@
                         throw new HyracksDataException(e);
                     }
                 }
-                int startOffset = 0;
-                if (offsetFieldIndex > 0)
-                    startOffset = tb.getFieldEndOffsets()[offsetFieldIndex - 1];
-                else
-                    startOffset = 0;
-                int endSize = tb.getSize();
-                int len = endSize - startSize;
-                aggregate(accessor, tIndex, tb.getByteArray(), startOffset, len);
-            }
 
-            @Override
-            public int aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset, int length)
-                    throws HyracksDataException {
+                // doing initial aggregate
                 ftr.reset(accessor, tIndex);
-                int start = offset;
                 for (int i = 0; i < aggs.length; i++) {
                     try {
+                        byte[] data = tb.getByteArray();
+                        int prevFieldPos = i + keys.length - 1;
+                        int start = prevFieldPos >= 0 ? tb.getFieldEndOffsets()[prevFieldPos] : 0;
                         aggs[i].step(ftr, data, start, stateFieldLength[i]);
-                        start += stateFieldLength[i];
                     } catch (AlgebricksException e) {
                         throw new HyracksDataException(e);
                     }
                 }
-                return OFFSET_INT_LENGTH;
             }
 
             @Override
-            public void outputPartialResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tb)
-                    throws HyracksDataException {
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor,
+                    int stateTupleIndex, AggregateState state) throws HyracksDataException {
+                ftr.reset(accessor, tIndex);
+                int stateTupleStart = stateAccessor.getTupleStartOffset(stateTupleIndex);
+                for (int i = 0; i < aggs.length; i++) {
+                    try {
+                        byte[] data = stateAccessor.getBuffer().array();
+                        int start = stateAccessor.getFieldStartOffset(stateTupleIndex, i + keys.length)
+                                + stateTupleStart;
+                        aggs[i].step(ftr, data, start, stateFieldLength[i]);
+                    } catch (AlgebricksException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+            }
+
+            @Override
+            public void outputPartialResult(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
                 byte[] data = accessor.getBuffer().array();
                 int startOffset = accessor.getTupleStartOffset(tIndex);
                 int aggFieldOffset = accessor.getFieldStartOffset(tIndex, offsetFieldIndex);
@@ -103,8 +113,8 @@
             }
 
             @Override
-            public void outputResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tb)
-                    throws HyracksDataException {
+            public void outputFinalResult(ArrayTupleBuilder tb, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
                 byte[] data = accessor.getBuffer().array();
                 int startOffset = accessor.getTupleStartOffset(tIndex);
                 int aggFieldOffset = accessor.getFieldStartOffset(tIndex, offsetFieldIndex);
@@ -123,6 +133,7 @@
 
             @Override
             public void reset() {
+
             }
 
             @Override
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
index 986f21c..1743dfc 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/aggreg/SimpleAlgebricksAccumulatingAggregatorFactory.java
@@ -17,56 +17,46 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunction;
 import edu.uci.ics.hyracks.algebricks.core.algebra.runtime.base.IAggregateFunctionFactory;
 import edu.uci.ics.hyracks.algebricks.core.api.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.utils.Pair;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregator;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
-public class SimpleAlgebricksAccumulatingAggregatorFactory implements IAccumulatingAggregatorFactory {
+public class SimpleAlgebricksAccumulatingAggregatorFactory implements IAggregatorDescriptorFactory {
 
     private static final long serialVersionUID = 1L;
     private IAggregateFunctionFactory[] aggFactories;
-    private int[] keys;
-    private int[] fdColumns;
 
     public SimpleAlgebricksAccumulatingAggregatorFactory(IAggregateFunctionFactory[] aggFactories, int[] keys,
             int[] fdColumns) {
         this.aggFactories = aggFactories;
-        this.keys = keys;
-        this.fdColumns = fdColumns;
     }
 
+    @SuppressWarnings("unchecked")
     @Override
-    public IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
-            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
+            RecordDescriptor outRecordDescriptor, int[] aggKeys, int[] partialKeys) throws HyracksDataException {
 
-        final IAggregateFunction[] agg = new IAggregateFunction[aggFactories.length];
-        final ArrayBackedValueStorage[] aggOutput = new ArrayBackedValueStorage[aggFactories.length];
-        for (int i = 0; i < agg.length; i++) {
-            aggOutput[i] = new ArrayBackedValueStorage();
-            try {
-                agg[i] = aggFactories[i].createAggregateFunction(aggOutput[i]);
-            } catch (AlgebricksException e) {
-                throw new HyracksDataException(e);
-            }
-        }
-
-        return new IAccumulatingAggregator() {
+        return new IAggregatorDescriptor() {
 
             private FrameTupleReference ftr = new FrameTupleReference();
-            private ArrayTupleBuilder tb = new ArrayTupleBuilder(keys.length + fdColumns.length + agg.length);
-            private boolean pending;
 
             @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
-                pending = false;
-                for (int i = 0; i < aggOutput.length; i++) {
+            public void init(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                Pair<ArrayBackedValueStorage[], IAggregateFunction[]> aggState = (Pair<ArrayBackedValueStorage[], IAggregateFunction[]>) state.state;
+                ArrayBackedValueStorage[] aggOutput = aggState.first;
+                IAggregateFunction[] agg = aggState.second;
+
+                // initialize aggregate functions
+                for (int i = 0; i < agg.length; i++) {
                     aggOutput[i].reset();
                     try {
                         agg[i].init();
@@ -74,18 +64,7 @@
                         throw new HyracksDataException(e);
                     }
                 }
-                tb.reset();
-                for (int i = 0; i < keys.length; ++i) {
-                    tb.addField(accessor, tIndex, keys[i]);
-                }
-                for (int i = 0; i < fdColumns.length; i++) {
-                    tb.addField(accessor, tIndex, fdColumns[i]);
-                }
 
-            }
-
-            @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
                 ftr.reset(accessor, tIndex);
                 for (int i = 0; i < agg.length; i++) {
                     try {
@@ -97,27 +76,68 @@
             }
 
             @Override
-            public boolean output(FrameTupleAppender appender, IFrameTupleAccessor accessor, int tIndex,
-                    int[] keyFieldIndexes) throws HyracksDataException {
-                if (!pending) {
-                    for (int i = 0; i < agg.length; i++) {
-                        try {
-                            agg[i].finish();
-                            tb.addField(aggOutput[i].getBytes(), aggOutput[i].getStartIndex(), aggOutput[i].getLength());
-                        } catch (AlgebricksException e) {
-                            throw new HyracksDataException(e);
-                        }
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor,
+                    int stateTupleIndex, AggregateState state) throws HyracksDataException {
+                Pair<ArrayBackedValueStorage[], IAggregateFunction[]> aggState = (Pair<ArrayBackedValueStorage[], IAggregateFunction[]>) state.state;
+                IAggregateFunction[] agg = aggState.second;
+                ftr.reset(accessor, tIndex);
+                for (int i = 0; i < agg.length; i++) {
+                    try {
+                        agg[i].step(ftr);
+                    } catch (AlgebricksException e) {
+                        throw new HyracksDataException(e);
                     }
                 }
-                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    pending = true;
-                    return false;
-                } else {
-                    return true;
+            }
+
+            @Override
+            public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                Pair<ArrayBackedValueStorage[], IAggregateFunction[]> aggState = (Pair<ArrayBackedValueStorage[], IAggregateFunction[]>) state.state;
+                ArrayBackedValueStorage[] aggOutput = aggState.first;
+                IAggregateFunction[] agg = aggState.second;
+                for (int i = 0; i < agg.length; i++) {
+                    try {
+                        agg[i].finish();
+                        tupleBuilder.addField(aggOutput[i].getBytes(), aggOutput[i].getStartIndex(),
+                                aggOutput[i].getLength());
+                    } catch (AlgebricksException e) {
+                        throw new HyracksDataException(e);
+                    }
                 }
             }
 
+            @Override
+            public AggregateState createAggregateStates() {
+                IAggregateFunction[] agg = new IAggregateFunction[aggFactories.length];
+                ArrayBackedValueStorage[] aggOutput = new ArrayBackedValueStorage[aggFactories.length];
+                for (int i = 0; i < agg.length; i++) {
+                    aggOutput[i] = new ArrayBackedValueStorage();
+                    try {
+                        agg[i] = aggFactories[i].createAggregateFunction(aggOutput[i]);
+                    } catch (AlgebricksException e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+                return new AggregateState(new Pair<ArrayBackedValueStorage[], IAggregateFunction[]>(aggOutput, agg));
+            }
+
+            @Override
+            public void reset() {
+
+            }
+
+            @Override
+            public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                throw new IllegalStateException("this method should not be called");
+            }
+
+            @Override
+            public void close() {
+
+            }
+
         };
     }
-
-}
+}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
index 237a903..b15d63d 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
@@ -14,8 +14,8 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregator;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupWriter;
 
 public class MicroPreClusteredGroupRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
@@ -23,12 +23,12 @@
     private static final long serialVersionUID = 1L;
     private final int[] groupFields;
     private final IBinaryComparatorFactory[] comparatorFactories;
-    private final IAccumulatingAggregatorFactory aggregatorFactory;
+    private final IAggregatorDescriptorFactory aggregatorFactory;
     private final RecordDescriptor inRecordDesc;
     private final RecordDescriptor outRecordDesc;
 
     public MicroPreClusteredGroupRuntimeFactory(int[] groupFields, IBinaryComparatorFactory[] comparatorFactories,
-            IAccumulatingAggregatorFactory aggregatorFactory, RecordDescriptor inRecordDesc,
+            IAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor inRecordDesc,
             RecordDescriptor outRecordDesc, int[] projectionList) {
         super(projectionList);
         // Obs: the projection list is currently ignored.
@@ -51,8 +51,8 @@
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
             }
             final IHyracksTaskContext ctx = context.getHyracksContext();
-            final IAccumulatingAggregator aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc,
-                    outRecordDesc);
+            final IAggregatorDescriptor aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc,
+                    outRecordDesc, groupFields, groupFields);
             final ByteBuffer copyFrame = ctx.allocateFrame();
             final FrameTupleAccessor copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
             copyFrameAccessor.reset(copyFrame);
@@ -66,7 +66,8 @@
 
                 @Override
                 public void open() throws HyracksDataException {
-                    pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregator, inRecordDesc, writer);
+                    pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregator, inRecordDesc,
+                            outRecordDesc, writer);
                     pgw.open();
                 }
 
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
index c550a3d..0370632 100644
--- a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
@@ -76,7 +76,7 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.LineFileWriteOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.HashGroupOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.SplitOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
@@ -492,7 +492,7 @@
         ITuplePartitionComputerFactory tpcf = new FieldHashPartitionComputerFactory(new int[] { 3 },
                 new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) });
         IAggregateFunctionFactory[] aggFuns = new IAggregateFunctionFactory[] { new TupleCountAggregateFunctionFactory() };
-        IAccumulatingAggregatorFactory aggFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(aggFuns,
+        IAggregatorDescriptorFactory aggFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(aggFuns,
                 new int[] { 3 }, new int[] {});
         HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(spec, new int[] { 3 }, tpcf,
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
index 7647e50..56ba785 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
@@ -59,6 +59,25 @@
         }
         return false;
     }
+    
+    public boolean appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length){
+        if (tupleDataEndOffset + fieldSlots.length * 4 + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
+            int effectiveSlots = 0;
+            for (int i = 0; i < fieldSlots.length; ++i) {
+                if(fieldSlots[i] > 0){
+                    buffer.putInt(tupleDataEndOffset + i * 4, fieldSlots[i]);
+                    effectiveSlots++;
+                }
+            }
+            System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset + effectiveSlots * 4, length);
+            tupleDataEndOffset += effectiveSlots * 4 + length;
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            ++tupleCount;
+            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+            return true;
+        }
+        return false;
+    }
 
     public boolean append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset) {
         int length = tEndOffset - tStartOffset;
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java
deleted file mode 100644
index 45d49e2..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-
-public class AvgAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
-    private static final long serialVersionUID = 1L;
-    private final int avgField;
-    private int outField = -1;
-
-    public AvgAggregatorDescriptorFactory(int avgField) {
-        this.avgField = avgField;
-    }
-
-    public AvgAggregatorDescriptorFactory(int avgField, int outField) {
-        this.avgField = avgField;
-        this.outField = outField;
-    }
-
-    @Override
-    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, int[] keyFields) throws HyracksDataException {
-
-        if (this.outField < 0)
-            this.outField = keyFields.length;
-
-        return new IAggregatorDescriptor() {
-
-            @Override
-            public void reset() {
-
-            }
-
-            @Override
-            public void outputResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, outField);
-                int sum = IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
-                int count = IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart + 4);
-
-                try {
-                    tupleBuilder.getDataOutput().writeInt(sum / count);
-                    tupleBuilder.addFieldEndOffset();
-                } catch (IOException e) {
-                    throw new HyracksDataException();
-                }
-            }
-
-            @Override
-            public void outputPartialResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, outField);
-                int sum = IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
-                int count = IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart + 4);
-                try {
-                    tupleBuilder.getDataOutput().writeInt(sum);
-                    tupleBuilder.getDataOutput().writeInt(count);
-                    tupleBuilder.addFieldEndOffset();
-                } catch (IOException e) {
-                    throw new HyracksDataException();
-                }
-            }
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                // Init aggregation value
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, avgField);
-                int sum = IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
-                int count = 1;
-
-                try {
-                    tupleBuilder.getDataOutput().writeInt(sum);
-                    tupleBuilder.getDataOutput().writeInt(count);
-                    tupleBuilder.addFieldEndOffset();
-                } catch (IOException e) {
-                    throw new HyracksDataException();
-                }
-            }
-
-            @Override
-            public void close() {
-
-            }
-
-            @Override
-            public int aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset, int length)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, outField);
-                int sum1 = IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
-                int count1 = 1;
-
-                int sum2 = IntegerSerializerDeserializer.getInt(data, offset);
-                int count2 = IntegerSerializerDeserializer.getInt(data, offset + 4);
-
-                ByteBuffer buf = ByteBuffer.wrap(data, offset, 8);
-                buf.putInt(sum1 + sum2);
-                buf.putInt(count1 + count2);
-
-                return 8;
-            }
-        };
-    }
-
-}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java
deleted file mode 100644
index 1cc3340..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-
-public class ConcatAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
-    private static final long serialVersionUID = 1L;
-    private static final int INIT_ACCUMULATORS_SIZE = 8;
-    private final int concatField;
-    private int outField = -1;
-
-    /**
-     * Initialize the aggregator, with the field to be concatenated.
-     * 
-     * @param concatField
-     */
-    public ConcatAggregatorDescriptorFactory(int concatField) {
-        this.concatField = concatField;
-    }
-
-    /**
-     * Initialize the aggregator, with the field index to be concatenated, and
-     * also the field where the aggregation result will be outputted.
-     * 
-     * @param concatField
-     * @param outField
-     */
-    public ConcatAggregatorDescriptorFactory(int concatField, int outField) {
-        this.concatField = concatField;
-        this.outField = outField;
-    }
-
-    /**
-     * Create a concatenation aggregator. A byte buffer will be allocated inside of the
-     * aggregator to contain the partial aggregation results. A reference will be written
-     * onto the output frame for indexing the aggregation result from the buffer.
-     */
-    @Override
-    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, final int[] keyFields) throws HyracksDataException {
-
-        if (this.outField < 0)
-            this.outField = keyFields.length;
-
-        return new IAggregatorDescriptor() {
-
-            byte[][] buf = new byte[INIT_ACCUMULATORS_SIZE][];
-
-            int currentAggregatorIndex = -1;
-            int aggregatorCount = 0;
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                // Initialize the aggregation value
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, concatField);
-                int fieldLength = accessor.getFieldLength(tIndex, concatField);
-                int appendOffset = tupleOffset + accessor.getFieldSlotsLength() + fieldStart;
-                // Get the initial value
-                currentAggregatorIndex++;
-                if (currentAggregatorIndex >= buf.length) {
-                    byte[][] newBuf = new byte[buf.length * 2][];
-                    for (int i = 0; i < buf.length; i++) {
-                        newBuf[i] = buf[i];
-                    }
-                    this.buf = newBuf;
-                }
-                buf[currentAggregatorIndex] = new byte[fieldLength];
-                System.arraycopy(accessor.getBuffer().array(), appendOffset, buf[currentAggregatorIndex], 0,
-                        fieldLength);
-                // Update the aggregator index
-                aggregatorCount++;
-
-                try {
-                    tupleBuilder.addField(IntegerSerializerDeserializer.INSTANCE, currentAggregatorIndex);
-                } catch (IOException e) {
-                    throw new HyracksDataException();
-                }
-            }
-
-            @Override
-            public void reset() {
-                currentAggregatorIndex = -1;
-                aggregatorCount = 0;
-            }
-
-            @Override
-            public void close() {
-                currentAggregatorIndex = -1;
-                aggregatorCount = 0;
-                for (int i = 0; i < buf.length; i++) {
-                    buf[i] = null;
-                }
-            }
-
-            @Override
-            public int aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset, int length)
-                    throws HyracksDataException {
-                int refIndex = IntegerSerializerDeserializer.getInt(data, offset);
-                // FIXME Should be done in binary way
-                StringBuilder sbder = new StringBuilder();
-                sbder.append(UTF8StringSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
-                        new ByteArrayInputStream(buf[refIndex]))));
-                // Get the new data
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, concatField);
-                int fieldLength = accessor.getFieldLength(tIndex, concatField);
-                sbder.append(UTF8StringSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
-                        new ByteArrayInputStream(accessor.getBuffer().array(), tupleOffset
-                                + accessor.getFieldSlotsLength() + fieldStart, fieldLength))));
-
-                ByteArrayOutputStream baos = new ByteArrayOutputStream();
-                UTF8StringSerializerDeserializer.INSTANCE.serialize(sbder.toString(), new DataOutputStream(baos));
-                buf[refIndex] = baos.toByteArray();
-                return 4;
-            }
-
-            @Override
-            public void outputResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, outField);
-                int refIndex = IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(), tupleOffset
-                        + accessor.getFieldSlotsLength() + fieldStart);
-
-                try {
-                    if (refIndex >= 0)
-                        tupleBuilder.getDataOutput().write(buf[refIndex]);
-                    else {
-                        int fieldLength = accessor.getFieldLength(tIndex, outField);
-                        tupleBuilder.getDataOutput().write(accessor.getBuffer().array(),
-                                tupleOffset + accessor.getFieldSlotsLength() + fieldStart + 4, fieldLength - 4);
-                    }
-                    tupleBuilder.addFieldEndOffset();
-                } catch (IOException e) {
-                    throw new HyracksDataException();
-                }
-            }
-
-            @Override
-            public void outputPartialResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldOffset = accessor.getFieldStartOffset(tIndex, outField);
-                int fieldLength = accessor.getFieldLength(tIndex, outField);
-                int refIndex = IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(), tupleOffset
-                        + accessor.getFieldSlotsLength() + fieldOffset);
-
-                try {
-                    tupleBuilder.getDataOutput().writeInt(-1);
-                    if (refIndex < 0) {
-                        tupleBuilder.getDataOutput().write(accessor.getBuffer().array(),
-                                tupleOffset + accessor.getFieldSlotsLength() + fieldOffset + 4, fieldLength - 4);
-                    } else {
-                        tupleBuilder.getDataOutput().write(buf[refIndex], 0, buf[refIndex].length);
-                    }
-                    tupleBuilder.addFieldEndOffset();
-                } catch (IOException e) {
-                    throw new HyracksDataException();
-                }
-            }
-        };
-    }
-}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java
deleted file mode 100644
index 74ac53a..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-
-public class CountAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
-    private static final long serialVersionUID = 1L;
-    private int outField = -1;
-
-    public CountAggregatorDescriptorFactory() {
-    }
-
-    public CountAggregatorDescriptorFactory(int outField) {
-        this.outField = outField;
-    }
-
-    @Override
-    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, final int[] keyFields) throws HyracksDataException {
-
-        if (this.outField < 0) {
-            this.outField = keyFields.length;
-        }
-        return new IAggregatorDescriptor() {
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                tupleBuilder.addField(IntegerSerializerDeserializer.INSTANCE, 1);
-            }
-
-            @Override
-            public void close() {
-            }
-
-            @Override
-            public int aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset, int length)
-                    throws HyracksDataException {
-                ByteBuffer buf = ByteBuffer.wrap(data);
-                int count = buf.getInt(offset);
-                buf.putInt(offset, count + 1);
-                return 4;
-            }
-
-            @Override
-            public void outputPartialResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, outField);
-
-                try {
-                    tupleBuilder.getDataOutput().write(accessor.getBuffer().array(),
-                            tupleOffset + accessor.getFieldSlotsLength() + fieldStart, 4);
-                    tupleBuilder.addFieldEndOffset();
-                } catch (IOException e) {
-                    throw new HyracksDataException("Failed to write int sum as a partial result.");
-                }
-            }
-
-            @Override
-            public void outputResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, outField);
-
-                try {
-                    tupleBuilder.getDataOutput().write(accessor.getBuffer().array(),
-                            tupleOffset + accessor.getFieldSlotsLength() + fieldStart, 4);
-                    tupleBuilder.addFieldEndOffset();
-                } catch (IOException e) {
-                    throw new HyracksDataException("Failed to write int sum as a partial result.");
-                }
-            }
-
-            @Override
-            public void reset() {
-
-            }
-        };
-    }
-
-}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorFactory.java
deleted file mode 100644
index f1e54e2..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorFactory.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-
-public class CountAggregatorFactory implements IFieldValueResultingAggregatorFactory {
-    private static final long serialVersionUID = 1L;
-
-    @Override
-    public IFieldValueResultingAggregator createFieldValueResultingAggregator() {
-        return new IFieldValueResultingAggregator() {
-            private int count;
-
-            @Override
-            public void output(DataOutput resultAcceptor) throws HyracksDataException {
-                try {
-                    resultAcceptor.writeInt(count);
-                } catch (IOException e) {
-                    throw new HyracksDataException(e);
-                }
-            }
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
-                count = 0;
-            }
-
-            @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
-                ++count;
-            }
-        };
-    }
-
-    @Override
-    public ISpillableFieldValueResultingAggregator createSpillableFieldValueResultingAggregator() {
-        return new ISpillableFieldValueResultingAggregator() {
-            private int count;
-
-            @Override
-            public void output(DataOutput resultAcceptor) throws HyracksDataException {
-                try {
-                    resultAcceptor.writeInt(count);
-                } catch (IOException e) {
-                    throw new HyracksDataException(e);
-                }
-            }
-
-            @Override
-            public void initFromPartial(IFrameTupleAccessor accessor, int tIndex, int fIndex)
-                    throws HyracksDataException {
-                count = IntegerSerializerDeserializer.getInt(
-                        accessor.getBuffer().array(),
-                        accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
-                                + accessor.getFieldStartOffset(tIndex, fIndex));
-
-            }
-
-            @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
-                count++;
-            }
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
-                count = 0;
-            }
-
-            @Override
-            public void accumulatePartialResult(IFrameTupleAccessor accessor, int tIndex, int fIndex)
-                    throws HyracksDataException {
-                count += IntegerSerializerDeserializer.getInt(
-                        accessor.getBuffer().array(),
-                        accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
-                                + accessor.getFieldStartOffset(tIndex, fIndex));
-            }
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/FloatSumAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/FloatSumAggregatorFactory.java
deleted file mode 100644
index 515cc21..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/FloatSumAggregatorFactory.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
-
-/**
- * SUM aggregator on float type data.
- * 
- */
-public class FloatSumAggregatorFactory implements
-        IFieldValueResultingAggregatorFactory {
-
-    /**
-	 * 
-	 */
-    private static final long serialVersionUID = 1L;
-    private int sumField;
-
-    public FloatSumAggregatorFactory(int field) {
-        this.sumField = field;
-    }
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see edu.uci.ics.hyracks.dataflow.std.aggregators.
-     * IFieldValueResultingAggregatorFactory
-     * #createFieldValueResultingAggregator()
-     */
-    @Override
-    public IFieldValueResultingAggregator createFieldValueResultingAggregator() {
-        return new IFieldValueResultingAggregator() {
-
-            private float sum;
-
-            @Override
-            public void output(DataOutput resultAcceptor)
-                    throws HyracksDataException {
-                try {
-                    resultAcceptor.writeFloat(sum);
-                } catch (IOException ex) {
-                    throw new HyracksDataException(ex);
-                }
-            }
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                sum = 0;
-            }
-
-            @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, sumField);
-                sum += FloatSerializerDeserializer.getFloat(accessor
-                        .getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength()
-                                + fieldStart);
-            }
-        };
-    }
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see edu.uci.ics.hyracks.dataflow.std.aggregators.
-     * IFieldValueResultingAggregatorFactory
-     * #createSpillableFieldValueResultingAggregator()
-     */
-    @Override
-    public ISpillableFieldValueResultingAggregator createSpillableFieldValueResultingAggregator() {
-        return new ISpillableFieldValueResultingAggregator() {
-
-            private float sum;
-
-            @Override
-            public void output(DataOutput resultAcceptor)
-                    throws HyracksDataException {
-                try {
-                    resultAcceptor.writeFloat(sum);
-                } catch (IOException ex) {
-                    throw new HyracksDataException(ex);
-                }
-            }
-
-            @Override
-            public void initFromPartial(IFrameTupleAccessor accessor,
-                    int tIndex, int fIndex) throws HyracksDataException {
-                sum = FloatSerializerDeserializer.getFloat(
-                        accessor.getBuffer().array(),
-                        accessor.getTupleStartOffset(tIndex)
-                                + accessor.getFieldSlotsLength()
-                                + accessor.getFieldStartOffset(tIndex, fIndex));
-            }
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                sum = 0;
-            }
-
-            @Override
-            public void accumulatePartialResult(IFrameTupleAccessor accessor,
-                    int tIndex, int fIndex) throws HyracksDataException {
-                sum += FloatSerializerDeserializer.getFloat(
-                        accessor.getBuffer().array(),
-                        accessor.getTupleStartOffset(tIndex)
-                                + accessor.getFieldSlotsLength()
-                                + accessor.getFieldStartOffset(tIndex, fIndex));
-            }
-
-            @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, sumField);
-                sum += FloatSerializerDeserializer.getFloat(accessor
-                        .getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength()
-                                + fieldStart);
-            }
-        };
-    }
-
-}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptor.java
deleted file mode 100644
index dfc31cd..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptor.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-public interface IAggregatorDescriptor {
-
-    /**
-     * Initialize the aggregator with an input tuple specified by the input
-     * frame and tuple index. This function will write the initialized partial
-     * result into the tuple builder.
-     * 
-     * @param accessor
-     * @param tIndex
-     * @param tupleBuilder
-     * @throws HyracksDataException
-     */
-    public void init(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-            throws HyracksDataException;
-
-    /**
-     * Aggregate the input tuple with the partial result specified by the bytes.
-     * The new value then is written back to the bytes field specified.
-     * It is the developer's responsibility to have the new result not exceed
-     * the given bytes.
-     * 
-     * @param accessor
-     * @param tIndex
-     * @param data
-     * @param offset
-     * @param length
-     * @return
-     * @throws HyracksDataException
-     */
-    public int aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset, int length)
-            throws HyracksDataException;
-
-    /**
-     * Output the partial aggregation result to an array tuple builder.
-     * Necessary additional information for aggregation should be maintained.
-     * For example, for an aggregator calculating AVG, the count and also the
-     * current average should be maintained as the partial results.
-     * 
-     * @param accessor
-     * @param tIndex
-     * @param tupleBuilder
-     * @throws HyracksDataException
-     */
-    public void outputPartialResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-            throws HyracksDataException;
-
-    /**
-     * Output the final aggregation result to an array tuple builder.
-     * 
-     * @param accessor
-     * @param tIndex
-     * @param tupleBuilder
-     * @return
-     * @throws HyracksDataException
-     */
-    public void outputResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-            throws HyracksDataException;
-
-    /**
-     * reset the internal states
-     */
-    public void reset();
-
-    /**
-     * Close the aggregator. Necessary clean-up code should be implemented here.
-     */
-    public void close();
-
-}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IFieldValueResultingAggregator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IFieldValueResultingAggregator.java
deleted file mode 100644
index 783467b..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IFieldValueResultingAggregator.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.DataOutput;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public interface IFieldValueResultingAggregator {
-    /**
-     * Called once per aggregator before calling accumulate for the first time.
-     * 
-     * @param accessor
-     *            - Accessor to the data tuple.
-     * @param tIndex
-     *            - Index of the tuple in the accessor.
-     * @throws HyracksDataException
-     */
-    public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
-
-    /**
-     * Called once per tuple that belongs to this group.
-     * 
-     * @param accessor
-     *            - Accessor to data tuple.
-     * @param tIndex
-     *            - Index of tuple in the accessor.
-     * @throws HyracksDataException
-     */
-    public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
-
-    /**
-     * Called finally to emit output.
-     * 
-     * @param resultAcceptor
-     *            - Interface to write the result to.
-     * @throws HyracksDataException
-     */
-    public void output(DataOutput resultAcceptor) throws HyracksDataException;
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IFieldValueResultingAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IFieldValueResultingAggregatorFactory.java
deleted file mode 100644
index 1b7da7f..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IFieldValueResultingAggregatorFactory.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.Serializable;
-
-public interface IFieldValueResultingAggregatorFactory extends Serializable {
-    public IFieldValueResultingAggregator createFieldValueResultingAggregator();
-
-    public ISpillableFieldValueResultingAggregator createSpillableFieldValueResultingAggregator();
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ISpillableFieldValueResultingAggregator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ISpillableFieldValueResultingAggregator.java
deleted file mode 100644
index d52f458..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ISpillableFieldValueResultingAggregator.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * An extended version of the {@link IFieldValueResultingAggregator} supporting
- * external aggregation.
- */
-public interface ISpillableFieldValueResultingAggregator extends IFieldValueResultingAggregator {
-
-    /**
-     * Called once per aggregator before calling accumulate for the first time.
-     * 
-     * @param accessor
-     *            - Accessor to the data tuple.
-     * @param tIndex
-     *            - Index of the tuple in the accessor.
-     * @throws HyracksDataException
-     */
-    public void initFromPartial(IFrameTupleAccessor accessor, int tIndex, int fIndex) throws HyracksDataException;
-
-    /**
-     * Aggregate another partial result.
-     * 
-     * @param accessor
-     * @param tIndex
-     * @param fIndex
-     * @throws HyracksDataException
-     */
-    public void accumulatePartialResult(IFrameTupleAccessor accessor, int tIndex, int fIndex)
-            throws HyracksDataException;
-
-}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java
deleted file mode 100644
index 770d36b..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-
-public class IntSumAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
-    private static final long serialVersionUID = 1L;
-    private final int aggField;
-    private int outField = -1;
-
-    public IntSumAggregatorDescriptorFactory(int aggField) {
-        this.aggField = aggField;
-    }
-
-    public IntSumAggregatorDescriptorFactory(int aggField, int outField) {
-        this.aggField = aggField;
-        this.outField = outField;
-    }
-
-    @Override
-    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, final int[] keyFields) throws HyracksDataException {
-
-        if (this.outField < 0) {
-            this.outField = keyFields.length;
-        }
-
-        return new IAggregatorDescriptor() {
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                int sum = 0;
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
-                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
-
-                tupleBuilder.addField(IntegerSerializerDeserializer.INSTANCE, sum);
-            }
-
-            @Override
-            public void close() {
-            }
-
-            @Override
-            public int aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset, int length)
-                    throws HyracksDataException {
-                int sum = 0;
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
-                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
-                // Update the value of tuple 2
-                ByteBuffer buf = ByteBuffer.wrap(data);
-                sum += buf.getInt(offset);
-                buf.putInt(offset, sum);
-                return 4;
-            }
-
-            @Override
-            public void outputPartialResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, outField);
-
-                try {
-                    tupleBuilder.getDataOutput().write(accessor.getBuffer().array(),
-                            tupleOffset + accessor.getFieldSlotsLength() + fieldStart, 4);
-                    tupleBuilder.addFieldEndOffset();
-                } catch (IOException e) {
-                    throw new HyracksDataException("Failed to write int sum as a partial result.");
-                }
-            }
-
-            @Override
-            public void outputResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, outField);
-
-                try {
-                    tupleBuilder.getDataOutput().write(accessor.getBuffer().array(),
-                            tupleOffset + accessor.getFieldSlotsLength() + fieldStart, 4);
-                    tupleBuilder.addFieldEndOffset();
-                } catch (IOException e) {
-                    throw new HyracksDataException("Failed to write int sum as a partial result.");
-                }
-            }
-
-            @Override
-            public void reset() {
-
-            }
-        };
-    }
-}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MinMaxAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MinMaxAggregatorFactory.java
deleted file mode 100644
index 0db3511..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MinMaxAggregatorFactory.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
-
-/**
- * Min/Max aggregator factory
- */
-public class MinMaxAggregatorFactory implements
-        IFieldValueResultingAggregatorFactory {
-
-    /**
-	 * 
-	 */
-    private static final long serialVersionUID = 1L;
-
-    /**
-     * indicate the type of the value: true: max false: min
-     */
-    private boolean type;
-
-    /**
-     * The field to be aggregated.
-     */
-    private int field;
-
-    public MinMaxAggregatorFactory(boolean type, int field) {
-        this.type = type;
-        this.field = field;
-    }
-
-    @Override
-    public IFieldValueResultingAggregator createFieldValueResultingAggregator() {
-        return new IFieldValueResultingAggregator() {
-
-            private float minmax;
-
-            @Override
-            public void output(DataOutput resultAcceptor)
-                    throws HyracksDataException {
-                try {
-                    resultAcceptor.writeFloat(minmax);
-                } catch (IOException ex) {
-                    throw new HyracksDataException(ex);
-                }
-            }
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                minmax = 0;
-            }
-
-            @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, field);
-                float nval = FloatSerializerDeserializer.getFloat(accessor
-                        .getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength()
-                                + fieldStart);
-                if ((type ? (nval > minmax) : (nval < minmax))) {
-                    minmax = nval;
-                }
-            }
-        };
-    }
-
-    @Override
-    public ISpillableFieldValueResultingAggregator createSpillableFieldValueResultingAggregator() {
-        return new ISpillableFieldValueResultingAggregator() {
-
-            private float minmax;
-
-            @Override
-            public void output(DataOutput resultAcceptor)
-                    throws HyracksDataException {
-                try {
-                    resultAcceptor.writeFloat(minmax);
-                } catch (IOException ex) {
-                    throw new HyracksDataException(ex);
-                }
-            }
-
-            @Override
-            public void initFromPartial(IFrameTupleAccessor accessor,
-                    int tIndex, int fIndex) throws HyracksDataException {
-                minmax = FloatSerializerDeserializer.getFloat(
-                        accessor.getBuffer().array(),
-                        accessor.getTupleStartOffset(tIndex)
-                                + accessor.getFieldSlotsLength()
-                                + accessor.getFieldStartOffset(tIndex, fIndex));
-
-            }
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                minmax = 0;
-            }
-
-            @Override
-            public void accumulatePartialResult(IFrameTupleAccessor accessor,
-                    int tIndex, int fIndex) throws HyracksDataException {
-                minmax = FloatSerializerDeserializer.getFloat(
-                        accessor.getBuffer().array(),
-                        accessor.getTupleStartOffset(tIndex)
-                                + accessor.getFieldSlotsLength()
-                                + accessor.getFieldStartOffset(tIndex, fIndex));
-
-            }
-
-            @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, field);
-                float nval = FloatSerializerDeserializer.getFloat(accessor
-                        .getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength()
-                                + fieldStart);
-                if ((type ? (nval > minmax) : (nval < minmax))) {
-                    minmax = nval;
-                }
-            }
-        };
-    }
-
-}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java
deleted file mode 100644
index 053b9e2..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-public class MultiAggregatorDescriptorFactory implements IAggregatorDescriptorFactory {
-    private static final long serialVersionUID = 1L;
-    private final IAggregatorDescriptorFactory[] aggregatorFactories;
-
-    public MultiAggregatorDescriptorFactory(IAggregatorDescriptorFactory[] aggregatorFactories) {
-        this.aggregatorFactories = aggregatorFactories;
-    }
-
-    @Override
-    public IAggregatorDescriptor createAggregator(final IHyracksTaskContext ctx,
-            final RecordDescriptor inRecordDescriptor, final RecordDescriptor outRecordDescriptor, final int[] keyFields)
-            throws HyracksDataException {
-
-        final IAggregatorDescriptor[] aggregators = new IAggregatorDescriptor[this.aggregatorFactories.length];
-        for (int i = 0; i < aggregators.length; i++) {
-            aggregators[i] = aggregatorFactories[i].createAggregator(ctx, inRecordDescriptor, outRecordDescriptor,
-                    keyFields);
-        }
-
-        return new IAggregatorDescriptor() {
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                for (int i = 0; i < aggregators.length; i++) {
-                    aggregators[i].init(accessor, tIndex, tupleBuilder);
-                }
-            }
-
-            @Override
-            public int aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset, int length)
-                    throws HyracksDataException {
-                int adjust = 0;
-                for (int i = 0; i < aggregators.length; i++) {
-                    adjust += aggregators[i].aggregate(accessor, tIndex, data, offset + adjust, length - adjust);
-                }
-                return adjust;
-            }
-
-            @Override
-            public void outputPartialResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                for (int i = 0; i < aggregators.length; i++) {
-                    aggregators[i].outputPartialResult(accessor, tIndex, tupleBuilder);
-                }
-            }
-
-            @Override
-            public void outputResult(IFrameTupleAccessor accessor, int tIndex, ArrayTupleBuilder tupleBuilder)
-                    throws HyracksDataException {
-                for (int i = 0; i < aggregators.length; i++) {
-                    aggregators[i].outputResult(accessor, tIndex, tupleBuilder);
-                }
-            }
-
-            @Override
-            public void close() {
-                for (int i = 0; i < aggregators.length; i++) {
-                    aggregators[i].close();
-                }
-            }
-
-            @Override
-            public void reset() {
-                for (int i = 0; i < aggregators.length; i++) {
-                    aggregators[i].reset();
-                }
-            }
-
-        };
-    }
-}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
deleted file mode 100644
index 4fec049..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.DataOutput;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregator;
-import edu.uci.ics.hyracks.dataflow.std.group.IAccumulatingAggregatorFactory;
-
-public class MultiAggregatorFactory implements IAccumulatingAggregatorFactory {
-    private static final long serialVersionUID = 1L;
-
-    private IFieldValueResultingAggregatorFactory[] aFactories;
-
-    public MultiAggregatorFactory(IFieldValueResultingAggregatorFactory[] aFactories) {
-        this.aFactories = aFactories;
-    }
-
-    @Override
-    public IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
-            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
-        final IFieldValueResultingAggregator aggregators[] = new IFieldValueResultingAggregator[aFactories.length];
-        for (int i = 0; i < aFactories.length; ++i) {
-            aggregators[i] = aFactories[i].createFieldValueResultingAggregator();
-        }
-        final ArrayTupleBuilder tb = new ArrayTupleBuilder(outRecordDescriptor.getFieldCount());
-        return new IAccumulatingAggregator() {
-
-            private boolean pending;
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
-                tb.reset();
-                for (int i = 0; i < aggregators.length; ++i) {
-                    aggregators[i].init(accessor, tIndex);
-                }
-                pending = false;
-            }
-
-            @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
-                for (int i = 0; i < aggregators.length; ++i) {
-                    aggregators[i].accumulate(accessor, tIndex);
-                }
-            }
-
-            @Override
-            public boolean output(FrameTupleAppender appender, IFrameTupleAccessor accessor, int tIndex,
-                    int[] keyFieldIndexes) throws HyracksDataException {
-                if (!pending) {
-                    tb.reset();
-                    for (int i = 0; i < keyFieldIndexes.length; ++i) {
-                        tb.addField(accessor, tIndex, keyFieldIndexes[i]);
-                    }
-                    DataOutput dos = tb.getDataOutput();
-                    for (int i = 0; i < aggregators.length; ++i) {
-                        aggregators[i].output(dos);
-                        tb.addFieldEndOffset();
-                    }
-                }
-                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    pending = true;
-                    return false;
-                }
-                return true;
-            }
-
-        };
-    }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumAggregatorFactory.java
deleted file mode 100644
index 0817263..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/SumAggregatorFactory.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-
-/**
- * SUM aggregator factory (for integer only; another SUM aggregator for floats
- * is available at {@link FloatSumAggregatorFactory})
- */
-public class SumAggregatorFactory implements
-        IFieldValueResultingAggregatorFactory {
-
-    private int sumField;
-
-    public SumAggregatorFactory(int field) {
-        sumField = field;
-    }
-
-    /**
-	 * 
-	 */
-    private static final long serialVersionUID = 1L;
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see edu.uci.ics.hyracks.dataflow.std.aggregators.
-     * IFieldValueResultingAggregatorFactory
-     * #createFieldValueResultingAggregator()
-     */
-    @Override
-    public IFieldValueResultingAggregator createFieldValueResultingAggregator() {
-        return new IFieldValueResultingAggregator() {
-            private int sum;
-
-            @Override
-            public void output(DataOutput resultAcceptor)
-                    throws HyracksDataException {
-                try {
-                    resultAcceptor.writeInt(sum);
-                } catch (IOException e) {
-                    throw new HyracksDataException(e);
-                }
-            }
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                sum = 0;
-            }
-
-            @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, sumField);
-                sum += IntegerSerializerDeserializer.getInt(accessor
-                        .getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength()
-                                + fieldStart);
-            }
-        };
-    }
-
-    /*
-     * (non-Javadoc)
-     * 
-     * @see edu.uci.ics.hyracks.dataflow.std.aggregators.spillable.
-     * ISpillableFieldValueResultingAggregatorFactory
-     * #createFieldValueResultingAggregator()
-     */
-    @Override
-    public ISpillableFieldValueResultingAggregator createSpillableFieldValueResultingAggregator() {
-        return new ISpillableFieldValueResultingAggregator() {
-
-            private int sum;
-
-            @Override
-            public void output(DataOutput resultAcceptor)
-                    throws HyracksDataException {
-                try {
-                    resultAcceptor.writeInt(sum);
-                } catch (IOException ex) {
-                    throw new HyracksDataException(ex);
-                }
-            }
-
-            @Override
-            public void initFromPartial(IFrameTupleAccessor accessor,
-                    int tIndex, int fIndex) throws HyracksDataException {
-                sum = IntegerSerializerDeserializer.getInt(
-                        accessor.getBuffer().array(),
-                        accessor.getTupleStartOffset(tIndex)
-                                + accessor.getFieldSlotsLength()
-                                + accessor.getFieldStartOffset(tIndex, fIndex));
-            }
-
-            @Override
-            public void init(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                sum = 0;
-            }
-
-            @Override
-            public void accumulatePartialResult(IFrameTupleAccessor accessor,
-                    int tIndex, int fIndex) throws HyracksDataException {
-                sum += IntegerSerializerDeserializer.getInt(
-                        accessor.getBuffer().array(),
-                        accessor.getTupleStartOffset(tIndex)
-                                + accessor.getFieldSlotsLength()
-                                + accessor.getFieldStartOffset(tIndex, fIndex));
-            }
-
-            @Override
-            public void accumulate(IFrameTupleAccessor accessor, int tIndex)
-                    throws HyracksDataException {
-                int tupleOffset = accessor.getTupleStartOffset(tIndex);
-                int fieldStart = accessor.getFieldStartOffset(tIndex, sumField);
-                sum += IntegerSerializerDeserializer.getInt(accessor
-                        .getBuffer().array(),
-                        tupleOffset + accessor.getFieldSlotsLength()
-                                + fieldStart);
-            }
-        };
-    }
-
-}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
index 000e459..1744b99 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
@@ -88,8 +88,8 @@
     protected abstract IRecordWriter createRecordWriter(FileSplit fileSplit, int index) throws Exception;
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            int partition, int nPartitions) {
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new FileWriteOperator(partition),
                 recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
     }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
index 0096576..7cf437d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
@@ -86,11 +86,7 @@
     }
 
     private enum State {
-        INIT,
-        IN_RECORD,
-        EOR,
-        CR,
-        EOF
+        INIT, IN_RECORD, EOR, CR, EOF
     }
 
     private class FieldCursor {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
index d85f437..76cb0ae 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
@@ -44,8 +44,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            int partition, int nPartitions) {
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         final FileSplit split = fileSplitProvider.getFileSplits()[partition];
         final ITupleParser tp = tupleParserFactory.createTupleParser(ctx);
         return new AbstractUnaryOutputSourceOperatorNodePushable() {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
index 7ce8594..8397d29 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
@@ -39,8 +39,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            final int partition, int nPartitions) {
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
         final FileSplit[] splits = fileSplitProvider.getFileSplits();
         return new AbstractUnaryInputSinkOperatorNodePushable() {
             private OutputStream out;
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
index 47a3158..6bf29d2 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
@@ -56,8 +56,15 @@
         this.delim = delim;
     }
 
-    /* (non-Javadoc)
-     * @see edu.uci.ics.hyracks.api.dataflow.IActivityNode#createPushRuntime(edu.uci.ics.hyracks.api.context.IHyracksContext, edu.uci.ics.hyracks.api.job.IOperatorEnvironment, edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider, int, int)
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.api.dataflow.IActivityNode#createPushRuntime(edu.
+     * uci.ics.hyracks.api.context.IHyracksContext,
+     * edu.uci.ics.hyracks.api.job.IOperatorEnvironment,
+     * edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider, int,
+     * int)
      */
     @Override
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordFileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordFileScanOperatorDescriptor.java
index 38ac1a3..af345aa 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordFileScanOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordFileScanOperatorDescriptor.java
@@ -69,11 +69,14 @@
 
     @Override
     protected void configure() throws Exception {
-        // currently a no-op, but is meant to initialize , if required before it is asked 
+        // currently a no-op, but is meant to initialize , if required before it
+        // is asked
         // to create a record reader
-        // this is executed at the node and is useful for operators that could not be 
-        // initialized from the client completely, because of lack of information specific 
-        // to the node where the operator gets executed. 
+        // this is executed at the node and is useful for operators that could
+        // not be
+        // initialized from the client completely, because of lack of
+        // information specific
+        // to the node where the operator gets executed.
 
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AggregateState.java
similarity index 61%
copy from hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
copy to hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AggregateState.java
index 978f671..e72f85c 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AggregateState.java
@@ -16,11 +16,29 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+/**
+ *
+ */
+public class AggregateState implements Serializable {
 
-public interface IAccumulatingAggregatorFactory extends Serializable {
-    IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
-            RecordDescriptor outRecordDescriptor) throws HyracksDataException;
-}
\ No newline at end of file
+    private static final long serialVersionUID = 1L;
+
+    public Object state = null;
+
+    public AggregateState() {
+        state = null;
+    }
+
+    public AggregateState(Object obj) {
+        state = obj;
+    }
+
+    public void reset() {
+        state = null;
+    }
+
+    public void close() {
+        state = null;
+    }
+
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/DeserializedPreclusteredGroupOperator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/DeserializedPreclusteredGroupOperator.java
index 4ac72f6..370f6d0 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/DeserializedPreclusteredGroupOperator.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/DeserializedPreclusteredGroupOperator.java
@@ -90,11 +90,6 @@
     }
 
     @Override
-    public void fail() throws HyracksDataException {
-        writer.fail();
-    }
-
-    @Override
     public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
         if (index != 0) {
             throw new IllegalArgumentException();
@@ -121,4 +116,10 @@
         }
         return 0;
     }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        // TODO Auto-generated method stub
+
+    }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
index 0680f69..41da49a 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
@@ -45,8 +45,6 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
 import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractTaskState;
@@ -55,7 +53,11 @@
 import edu.uci.ics.hyracks.dataflow.std.util.ReferenceEntry;
 import edu.uci.ics.hyracks.dataflow.std.util.ReferencedPriorityQueue;
 
+/**
+ *
+ */
 public class ExternalGroupOperatorDescriptor extends AbstractOperatorDescriptor {
+
     private static final int AGGREGATE_ACTIVITY_ID = 0;
 
     private static final int MERGE_ACTIVITY_ID = 1;
@@ -64,15 +66,17 @@
     private final int[] keyFields;
     private final IBinaryComparatorFactory[] comparatorFactories;
     private final INormalizedKeyComputerFactory firstNormalizerFactory;
+
     private final IAggregatorDescriptorFactory aggregatorFactory;
-    private final IAggregatorDescriptorFactory mergeFactory;
+    private final IAggregatorDescriptorFactory mergerFactory;
+
     private final int framesLimit;
     private final ISpillableTableFactory spillableTableFactory;
     private final boolean isOutputSorted;
 
     public ExternalGroupOperatorDescriptor(JobSpecification spec, int[] keyFields, int framesLimit,
             IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory firstNormalizerFactory,
-            IAggregatorDescriptorFactory aggregatorFactory, IAggregatorDescriptorFactory mergeFactory,
+            IAggregatorDescriptorFactory aggregatorFactory, IAggregatorDescriptorFactory mergerFactory,
             RecordDescriptor recordDescriptor, ISpillableTableFactory spillableTableFactory, boolean isOutputSorted) {
         super(spec, 1, 1);
         this.framesLimit = framesLimit;
@@ -83,9 +87,8 @@
              */
             throw new IllegalStateException("frame limit should at least be 2, but it is " + framesLimit + "!");
         }
-
         this.aggregatorFactory = aggregatorFactory;
-        this.mergeFactory = mergeFactory;
+        this.mergerFactory = mergerFactory;
         this.keyFields = keyFields;
         this.comparatorFactories = comparatorFactories;
         this.firstNormalizerFactory = firstNormalizerFactory;
@@ -93,13 +96,19 @@
         this.isOutputSorted = isOutputSorted;
 
         /**
-         * Set the record descriptor. Note that since
-         * this operator is a unary operator,
-         * only the first record descriptor is used here.
+         * Set the record descriptor. Note that since this operator is a unary
+         * operator, only the first record descriptor is used here.
          */
         recordDescriptors[0] = recordDescriptor;
     }
 
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor#contributeActivities
+     * (edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder)
+     */
     @Override
     public void contributeActivities(IActivityGraphBuilder builder) {
         AggregateActivity aggregateAct = new AggregateActivity(new ActivityId(getOperatorId(), AGGREGATE_ACTIVITY_ID));
@@ -185,7 +194,7 @@
 
                 @Override
                 public void fail() throws HyracksDataException {
-
+                    throw new HyracksDataException("failed");
                 }
 
                 @Override
@@ -244,8 +253,16 @@
             for (int i = 0; i < comparatorFactories.length; ++i) {
                 comparators[i] = comparatorFactories[i].createBinaryComparator();
             }
-            final IAggregatorDescriptor currentWorkingAggregator = mergeFactory.createAggregator(ctx,
-                    recordDescriptors[0], recordDescriptors[0], keyFields);
+
+            int[] keyFieldsInPartialResults = new int[keyFields.length];
+            for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
+                keyFieldsInPartialResults[i] = i;
+            }
+
+            final IAggregatorDescriptor aggregator = mergerFactory.createAggregator(ctx, recordDescriptors[0],
+                    recordDescriptors[0], keyFields, keyFieldsInPartialResults);
+            final AggregateState aggregateState = aggregator.createAggregateStates();
+
             final int[] storedKeys = new int[keyFields.length];
             /**
              * Get the list of the fields in the stored records.
@@ -253,10 +270,8 @@
             for (int i = 0; i < keyFields.length; ++i) {
                 storedKeys[i] = i;
             }
-            /**
-             * Tuple builder
-             */
-            final ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(recordDescriptors[0].getFieldCount());
+
+            final ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(recordDescriptors[0].getFields().length);
 
             IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
                 /**
@@ -268,11 +283,15 @@
                  * Output frame.
                  */
                 private ByteBuffer outFrame, writerFrame;
+                private final FrameTupleAppender outAppender = new FrameTupleAppender(ctx.getFrameSize());
+                private FrameTupleAppender writerAppender;
 
                 private LinkedList<RunFileReader> runs;
 
                 private AggregateActivityState aggState;
 
+                private ArrayTupleBuilder finalTupleBuilder;
+
                 /**
                  * how many frames to be read ahead once
                  */
@@ -280,11 +299,9 @@
 
                 private int[] currentFrameIndexInRun;
                 private int[] currentRunFrames;
-                private final FrameTupleAppender outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
+
                 private final FrameTupleAccessor outFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
                         recordDescriptors[0]);
-                private ArrayTupleBuilder finalTupleBuilder;
-                private FrameTupleAppender writerFrameAppender;
 
                 public void initialize() throws HyracksDataException {
                     aggState = (AggregateActivityState) ctx.getTaskState(new TaskId(new ActivityId(getOperatorId(),
@@ -299,11 +316,16 @@
                                     gTable.sortFrames();
                                 gTable.flushFrames(writer, false);
                             }
+                            gTable = null;
+                            aggState = null;
+                            System.gc();
                         } else {
+                            aggState = null;
+                            System.gc();
                             runs = new LinkedList<RunFileReader>(runs);
                             inFrames = new ArrayList<ByteBuffer>();
                             outFrame = ctx.allocateFrame();
-                            outFrameAppender.reset(outFrame, true);
+                            outAppender.reset(outFrame, true);
                             outFrameAccessor.reset(outFrame);
                             while (runs.size() > 0) {
                                 try {
@@ -318,6 +340,7 @@
                         writer.fail();
                         throw new HyracksDataException(e);
                     } finally {
+                        aggregateState.close();
                         writer.close();
                     }
                 }
@@ -346,8 +369,8 @@
                         currentFrameIndexInRun = new int[runNumber];
                         currentRunFrames = new int[runNumber];
                         /**
-                         * Create file readers for each input run file, only
-                         * for the ones fit into the inFrames
+                         * 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()];
@@ -378,6 +401,7 @@
                                         setNextTopTuple(runIndex, tupleIndices, runFileReaders, tupleAccessors,
                                                 topTuples);
                                 } else {
+                                    closeRun(runIndex, runFileReaders, tupleAccessors);
                                     break;
                                 }
                             }
@@ -399,47 +423,50 @@
                             if (currentTupleInOutFrame < 0
                                     || compareFrameTuples(fta, tupleIndex, outFrameAccessor, currentTupleInOutFrame) != 0) {
                                 /**
-                                 * Initialize the first output record
-                                 * Reset the tuple builder
+                                 * Initialize the first output record Reset the
+                                 * tuple builder
                                  */
+
                                 tupleBuilder.reset();
-                                for (int i = 0; i < keyFields.length; i++) {
-                                    tupleBuilder.addField(fta, tupleIndex, i);
+                                
+                                for(int k = 0; k < storedKeys.length; k++){
+                                	tupleBuilder.addField(fta, tupleIndex, storedKeys[k]);
                                 }
 
-                                currentWorkingAggregator.init(fta, tupleIndex, tupleBuilder);
-                                if (!outFrameAppender.append(tupleBuilder.getFieldEndOffsets(),
+                                aggregator.init(tupleBuilder, fta, tupleIndex, aggregateState);
+
+                                if (!outAppender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(),
                                         tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
                                     flushOutFrame(writer, finalPass);
-                                    if (!outFrameAppender.append(tupleBuilder.getFieldEndOffsets(),
-                                            tupleBuilder.getByteArray(), 0, tupleBuilder.getSize()))
+                                    if (!outAppender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(),
+                                            tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
                                         throw new HyracksDataException(
-                                                "Failed to append an aggregation result to the output frame.");
+                                                "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
+                                 * current aggregator do merge and output to the
+                                 * outFrame
                                  */
-                                int tupleOffset = outFrameAccessor.getTupleStartOffset(currentTupleInOutFrame);
-                                int fieldOffset = outFrameAccessor.getFieldStartOffset(currentTupleInOutFrame,
-                                        keyFields.length);
-                                int fieldLength = outFrameAccessor.getFieldLength(currentTupleInOutFrame,
-                                        keyFields.length);
-                                currentWorkingAggregator.aggregate(fta, tupleIndex, outFrameAccessor.getBuffer()
-                                        .array(), tupleOffset + outFrameAccessor.getFieldSlotsLength() + fieldOffset,
-                                        fieldLength);
+
+                                aggregator.aggregate(fta, tupleIndex, outFrameAccessor, currentTupleInOutFrame,
+                                        aggregateState);
+
                             }
                             tupleIndices[runIndex]++;
                             setNextTopTuple(runIndex, tupleIndices, runFileReaders, tupleAccessors, topTuples);
                         }
 
-                        if (outFrameAppender.getTupleCount() > 0) {
+                        if (outAppender.getTupleCount() > 0) {
                             flushOutFrame(writer, finalPass);
+                            outAppender.reset(outFrame, true);
                         }
 
-                        currentWorkingAggregator.close();
+                        aggregator.close();
+
                         runs.subList(0, runNumber).clear();
                         /**
                          * insert the new run file into the beginning of the run
@@ -456,42 +483,56 @@
                 }
 
                 private void flushOutFrame(IFrameWriter writer, boolean isFinal) throws HyracksDataException {
+
                     if (finalTupleBuilder == null) {
-                        finalTupleBuilder = new ArrayTupleBuilder(recordDescriptors[0].getFieldCount());
+                        finalTupleBuilder = new ArrayTupleBuilder(recordDescriptors[0].getFields().length);
                     }
+
                     if (writerFrame == null) {
                         writerFrame = ctx.allocateFrame();
                     }
-                    if (writerFrameAppender == null) {
-                        writerFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
-                        writerFrameAppender.reset(writerFrame, true);
-                    }
-                    outFrameAccessor.reset(outFrame);
-                    for (int i = 0; i < outFrameAccessor.getTupleCount(); i++) {
-                        finalTupleBuilder.reset();
-                        for (int j = 0; j < keyFields.length; j++) {
-                            finalTupleBuilder.addField(outFrameAccessor, i, j);
-                        }
-                        if (isFinal)
-                            currentWorkingAggregator.outputResult(outFrameAccessor, i, finalTupleBuilder);
-                        else
-                            currentWorkingAggregator.outputPartialResult(outFrameAccessor, i, finalTupleBuilder);
 
-                        if (!writerFrameAppender.append(finalTupleBuilder.getFieldEndOffsets(),
+                    if (writerAppender == null) {
+                        writerAppender = new FrameTupleAppender(ctx.getFrameSize());
+                        writerAppender.reset(writerFrame, true);
+                    }
+
+                    outFrameAccessor.reset(outFrame);
+
+                    for (int i = 0; i < outFrameAccessor.getTupleCount(); i++) {
+
+                        finalTupleBuilder.reset();
+
+                        for (int k = 0; k < storedKeys.length; k++) {
+                            finalTupleBuilder.addField(outFrameAccessor, i, storedKeys[k]);
+                        }
+
+                        if (isFinal) {
+
+                            aggregator.outputFinalResult(finalTupleBuilder, outFrameAccessor, i, aggregateState);
+
+                        } else {
+
+                            aggregator.outputPartialResult(finalTupleBuilder, outFrameAccessor, i, aggregateState);
+                        }
+
+                        if (!writerAppender.appendSkipEmptyField(finalTupleBuilder.getFieldEndOffsets(),
                                 finalTupleBuilder.getByteArray(), 0, finalTupleBuilder.getSize())) {
                             FrameUtils.flushFrame(writerFrame, writer);
-                            writerFrameAppender.reset(writerFrame, true);
-                            if (!writerFrameAppender.append(finalTupleBuilder.getFieldEndOffsets(),
-                                    finalTupleBuilder.getByteArray(), 0, finalTupleBuilder.getSize()))
+                            writerAppender.reset(writerFrame, true);
+                            if (!writerAppender.appendSkipEmptyField(finalTupleBuilder.getFieldEndOffsets(),
+                                    finalTupleBuilder.getByteArray(), 0, finalTupleBuilder.getSize())) {
                                 throw new HyracksDataException(
-                                        "Failed to write final aggregation result to a writer frame!");
+                                        "Aggregation output is too large to be fit into a frame.");
+                            }
                         }
                     }
-                    if (writerFrameAppender.getTupleCount() > 0) {
+                    if (writerAppender.getTupleCount() > 0) {
                         FrameUtils.flushFrame(writerFrame, writer);
-                        writerFrameAppender.reset(writerFrame, true);
+                        writerAppender.reset(writerFrame, true);
                     }
-                    outFrameAppender.reset(outFrame, true);
+
+                    outAppender.reset(outFrame, true);
                 }
 
                 private void setNextTopTuple(int runIndex, int[] tupleIndices, RunFileReader[] runCursors,
@@ -570,6 +611,7 @@
                     if (runCursors[index] != null) {
                         runCursors[index].close();
                         runCursors[index] = null;
+                        tupleAccessor[index] = null;
                     }
                 }
 
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/FrameToolsForGroupers.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/FrameToolsForGroupers.java
new file mode 100644
index 0000000..2346160
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/FrameToolsForGroupers.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.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-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
index 5733c11..43bb0ae 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
@@ -28,13 +28,15 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
 
 class GroupingHashTable {
     /**
-     * The pointers in the link store 3 int values for each entry in the hashtable: (bufferIdx, tIndex, accumulatorIdx).
+     * The pointers in the link store 3 int values for each entry in the
+     * hashtable: (bufferIdx, tIndex, accumulatorIdx).
      * 
      * @author vinayakb
      */
@@ -59,55 +61,82 @@
         }
     }
 
-    private static final int INIT_ACCUMULATORS_SIZE = 8;
+    private static final int INIT_AGG_STATE_SIZE = 8;
     private final IHyracksTaskContext ctx;
-    private final FrameTupleAppender appender;
+
     private final List<ByteBuffer> buffers;
     private final Link[] table;
-    private IAccumulatingAggregator[] accumulators;
+    /**
+     * 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[] fields;
     private final int[] storedKeys;
+    private final int[] keys;
     private final IBinaryComparator[] comparators;
     private final FrameTuplePairComparator ftpc;
     private final ITuplePartitionComputer tpc;
-    private final IAccumulatingAggregatorFactory aggregatorFactory;
-    private final RecordDescriptor inRecordDescriptor;
-    private final RecordDescriptor outRecordDescriptor;
+    private final IAggregatorDescriptor aggregator;
+
+    private final FrameTupleAppender appender;
 
     private final FrameTupleAccessor storedKeysAccessor;
 
+    private final ArrayTupleBuilder stateTupleBuilder, outputTupleBuilder;
+
     GroupingHashTable(IHyracksTaskContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
-            ITuplePartitionComputerFactory tpcf, IAccumulatingAggregatorFactory aggregatorFactory,
-            RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int tableSize) {
+            ITuplePartitionComputerFactory tpcf, IAggregatorDescriptorFactory aggregatorFactory,
+            RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int tableSize)
+            throws HyracksDataException {
         this.ctx = ctx;
-        appender = new FrameTupleAppender(ctx.getFrameSize());
+
         buffers = new ArrayList<ByteBuffer>();
         table = new Link[tableSize];
-        accumulators = new IAccumulatingAggregator[INIT_ACCUMULATORS_SIZE];
-        accumulatorSize = 0;
-        this.fields = fields;
+
+        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();
-        this.aggregatorFactory = aggregatorFactory;
-        this.inRecordDescriptor = inRecordDescriptor;
-        this.outRecordDescriptor = outRecordDescriptor;
+
+        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);
+
+        this.aggregateStates = new AggregateState[INIT_AGG_STATE_SIZE];
+        accumulatorSize = 0;
+
         RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(storedKeySerDeser);
         storedKeysAccessor = new FrameTupleAccessor(ctx.getFrameSize(), storedKeysRecordDescriptor);
         lastBIndex = -1;
+
+        appender = new FrameTupleAppender(ctx.getFrameSize());
+
         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);
     }
 
     private void addNewBuffer() {
@@ -119,57 +148,64 @@
         ++lastBIndex;
     }
 
-    private void flushFrame(FrameTupleAppender appender, IFrameWriter writer) throws HyracksDataException {
-        ByteBuffer frame = appender.getBuffer();
-        frame.position(0);
-        frame.limit(frame.capacity());
-        writer.nextFrame(appender.getBuffer());
-        appender.reset(appender.getBuffer(), true);
-    }
-
-    void insert(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+    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();
         }
-        IAccumulatingAggregator aggregator = null;
+        int saIndex = -1;
         for (int i = 0; i < link.size; i += 3) {
             int sbIndex = link.pointers[i];
             int stIndex = link.pointers[i + 1];
-            int saIndex = link.pointers[i + 2];
             storedKeysAccessor.reset(buffers.get(sbIndex));
             int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
             if (c == 0) {
-                aggregator = accumulators[saIndex];
+                saIndex = link.pointers[i + 2];
                 break;
             }
         }
-        if (aggregator == null) {
+        if (saIndex < 0) {
             // Did not find the key. Insert a new entry.
-            if (!appender.appendProjection(accessor, tIndex, fields)) {
+            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.appendProjection(accessor, tIndex, fields)) {
-                    throw new IllegalStateException();
+                if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
+                        stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
+                    throw new HyracksDataException("Cannot init the aggregate state in a single frame.");
                 }
             }
-            int sbIndex = lastBIndex;
-            int stIndex = appender.getTupleCount() - 1;
-            if (accumulatorSize >= accumulators.length) {
-                accumulators = Arrays.copyOf(accumulators, accumulators.length * 2);
+
+            if (accumulatorSize >= aggregateStates.length) {
+                aggregateStates = Arrays.copyOf(aggregateStates, aggregateStates.length * 2);
             }
-            int saIndex = accumulatorSize++;
-            aggregator = accumulators[saIndex] = aggregatorFactory.createAggregator(ctx, inRecordDescriptor,
-                    outRecordDescriptor);
-            aggregator.init(accessor, tIndex);
-            link.add(sbIndex, stIndex, saIndex);
+
+            aggregateStates[saIndex] = newState;
+
+            link.add(lastBIndex, appender.getTupleCount() - 1, saIndex);
+
+        } else {
+            aggregator.aggregate(accessor, tIndex, null, 0, aggregateStates[saIndex]);
         }
-        aggregator.accumulate(accessor, tIndex);
     }
 
     void write(IFrameWriter writer) throws HyracksDataException {
         ByteBuffer buffer = ctx.allocateFrame();
         appender.reset(buffer, true);
+
         for (int i = 0; i < table.length; ++i) {
             Link link = table[i];
             if (link != null) {
@@ -179,15 +215,37 @@
                     int aIndex = link.pointers[j + 2];
                     ByteBuffer keyBuffer = buffers.get(bIndex);
                     storedKeysAccessor.reset(keyBuffer);
-                    IAccumulatingAggregator aggregator = accumulators[aIndex];
-                    while (!aggregator.output(appender, storedKeysAccessor, tIndex, storedKeys)) {
-                        flushFrame(appender, writer);
+
+                    // 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]);
+
+                    if (!appender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
+                            outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
+                        writer.nextFrame(buffer);
+                        appender.reset(buffer, true);
+                        if (!appender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
+                                outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
+                            throw new HyracksDataException("Cannot write the aggregation output into a frame.");
+                        }
+                    }
+
                 }
             }
         }
         if (appender.getTupleCount() != 0) {
-            flushFrame(appender, writer);
+            writer.nextFrame(buffer);
+        }
+    }
+
+    void close() throws HyracksDataException {
+        for (AggregateState aState : aggregateStates) {
+            aState.close();
         }
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
index d0cd895..49443d1 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
@@ -38,7 +38,11 @@
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 
+/**
+ *
+ */
 public class HashGroupOperatorDescriptor extends AbstractOperatorDescriptor {
+
     private static final int HASH_BUILD_ACTIVITY_ID = 0;
 
     private static final int OUTPUT_ACTIVITY_ID = 1;
@@ -48,21 +52,30 @@
     private final int[] keys;
     private final ITuplePartitionComputerFactory tpcf;
     private final IBinaryComparatorFactory[] comparatorFactories;
-    private final IAccumulatingAggregatorFactory aggregatorFactory;
+
+    private final IAggregatorDescriptorFactory aggregatorFactory;
+
     private final int tableSize;
 
     public HashGroupOperatorDescriptor(JobSpecification spec, int[] keys, ITuplePartitionComputerFactory tpcf,
-            IBinaryComparatorFactory[] comparatorFactories, IAccumulatingAggregatorFactory aggregatorFactory,
-            RecordDescriptor recordDescriptor, int tableSize) {
+            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] = recordDescriptor;
+        recordDescriptors[0] = outRecordDescriptor;
         this.tableSize = tableSize;
     }
 
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor#contributeActivities
+     * (edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder)
+     */
     @Override
     public void contributeActivities(IActivityGraphBuilder builder) {
         HashBuildActivity ha = new HashBuildActivity(new ActivityId(odId, HASH_BUILD_ACTIVITY_ID));
@@ -105,8 +118,8 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IRecordDescriptorProvider recordDescProvider,
-                final int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
             final FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(),
                     recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
             return new AbstractUnaryInputSinkOperatorNodePushable() {
@@ -126,7 +139,12 @@
                     accessor.reset(buffer);
                     int tupleCount = accessor.getTupleCount();
                     for (int i = 0; i < tupleCount; ++i) {
-                        state.table.insert(accessor, i);
+                        try {
+                            state.table.insert(accessor, i);
+                        } catch (Exception e) {
+                            System.out.println(e.toString());
+                            throw new HyracksDataException(e);
+                        }
                     }
                 }
 
@@ -137,6 +155,7 @@
 
                 @Override
                 public void fail() throws HyracksDataException {
+                    throw new HyracksDataException("HashGroupOperator is failed.");
                 }
             };
         }
@@ -150,8 +169,8 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-                final int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
             return new AbstractUnaryOutputSourceOperatorNodePushable() {
                 @Override
                 public void initialize() throws HyracksDataException {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
similarity index 71%
rename from hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java
rename to hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index 32f94fe..b0e092d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -35,28 +35,42 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.structures.ISerializableTable;
 import edu.uci.ics.hyracks.dataflow.std.structures.SerializableHashTable;
 import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
 
-public class HashSpillableGroupingTableFactory implements ISpillableTableFactory {
+/**
+ *
+ */
+public class HashSpillableTableFactory implements ISpillableTableFactory {
+
     private static final long serialVersionUID = 1L;
     private final ITuplePartitionComputerFactory tpcf;
     private final int tableSize;
 
-    public HashSpillableGroupingTableFactory(ITuplePartitionComputerFactory tpcf, int tableSize) {
+    public HashSpillableTableFactory(ITuplePartitionComputerFactory tpcf, int tableSize) {
         this.tpcf = tpcf;
         this.tableSize = tableSize;
     }
 
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.dataflow.std.aggregations.ISpillableTableFactory#
+     * buildSpillableTable(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * int[], edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory[],
+     * edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory,
+     * edu.
+     * uci.ics.hyracks.dataflow.std.aggregations.IFieldAggregateDescriptorFactory
+     * [], edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int)
+     */
     @Override
     public ISpillableTable buildSpillableTable(final IHyracksTaskContext ctx, final int[] keyFields,
-            final IBinaryComparatorFactory[] comparatorFactories,
-            final INormalizedKeyComputerFactory firstKeyNormalizerFactory,
-            final IAggregatorDescriptorFactory aggregateDescriptorFactory, final RecordDescriptor inRecordDescriptor,
-            final RecordDescriptor outRecordDescriptor, final int framesLimit) throws HyracksDataException {
+            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];
@@ -92,28 +106,45 @@
         }
 
         final FrameTuplePairComparator ftpcPartial = new FrameTuplePairComparator(keyFields, storedKeys, comparators);
-        final FrameTuplePairComparator ftpcTuple = new FrameTuplePairComparator(storedKeys, storedKeys, comparators);
-        final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-        final ITuplePartitionComputer tpc = tpcf.createPartitioner();
-        final ByteBuffer outFrame = ctx.allocateFrame();
 
-        final ArrayTupleBuilder internalTupleBuilder;
-        if (keyFields.length < outRecordDescriptor.getFields().length)
-            internalTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
-        else
-            internalTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length + 1);
-        final ArrayTupleBuilder outputTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+        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 IAggregatorDescriptor aggregator = aggregateFactory.createAggregator(ctx, inRecordDescriptor,
+                outRecordDescriptor, keyFields, keyFieldsInPartialResults);
+
+        final AggregateState aggregateState = aggregator.createAggregateStates();
+
+        final ArrayTupleBuilder stateTupleBuilder;
+        if (keyFields.length < outRecordDescriptor.getFields().length) {
+            stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+        } else {
+            stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length + 1);
+        }
+
+        final ArrayTupleBuilder outputTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+
         return new ISpillableTable() {
-            private int dataFrameCount;
-            private final ISerializableTable table = new SerializableHashTable(tableSize, ctx);;
+
+            private int lastBufIndex;
+
+            private ByteBuffer outputFrame;
+            private FrameTupleAppender outputAppender;
+
+            private FrameTupleAppender stateAppender = new FrameTupleAppender(ctx.getFrameSize());
+
+            private final ISerializableTable table = new SerializableHashTable(tableSize, ctx);
             private final TuplePointer storedTuplePointer = new TuplePointer();
             private final List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
-            private int groupSize = 0;
-            private IAggregatorDescriptor aggregator = aggregateDescriptorFactory.createAggregator(ctx,
-                    inRecordDescriptor, outRecordDescriptor, keyFields);
 
             /**
              * A tuple is "pointed" to by 3 entries in the tPointers array. [0]
@@ -123,189 +154,6 @@
             private int[] tPointers;
 
             @Override
-            public void reset() {
-                groupSize = 0;
-                dataFrameCount = -1;
-                tPointers = null;
-                table.reset();
-                aggregator.close();
-            }
-
-            @Override
-            public boolean insert(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
-                if (dataFrameCount < 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));
-                    int c = ftpcPartial.compare(accessor, tIndex, storedKeysAccessor1, storedTuplePointer.tupleIndex);
-                    if (c == 0) {
-                        foundGroup = true;
-                        break;
-                    }
-                } while (true);
-
-                if (!foundGroup) {
-                    /**
-                     * If no matching group is found, create a new aggregator
-                     * Create a tuple for the new group
-                     */
-                    internalTupleBuilder.reset();
-                    for (int i = 0; i < keyFields.length; i++) {
-                        internalTupleBuilder.addField(accessor, tIndex, keyFields[i]);
-                    }
-                    aggregator.init(accessor, tIndex, internalTupleBuilder);
-                    if (!appender.append(internalTupleBuilder.getFieldEndOffsets(),
-                            internalTupleBuilder.getByteArray(), 0, internalTupleBuilder.getSize())) {
-                        if (!nextAvailableFrame()) {
-                            return false;
-                        } else {
-                            if (!appender.append(internalTupleBuilder.getFieldEndOffsets(),
-                                    internalTupleBuilder.getByteArray(), 0, internalTupleBuilder.getSize())) {
-                                throw new IllegalStateException("Failed to init an aggregator");
-                            }
-                        }
-                    }
-
-                    storedTuplePointer.frameIndex = dataFrameCount;
-                    storedTuplePointer.tupleIndex = appender.getTupleCount() - 1;
-                    table.insert(entry, storedTuplePointer);
-                    groupSize++;
-                } else {
-                    // If there is a matching found, do aggregation directly
-                    int tupleOffset = storedKeysAccessor1.getTupleStartOffset(storedTuplePointer.tupleIndex);
-                    int aggFieldOffset = storedKeysAccessor1.getFieldStartOffset(storedTuplePointer.tupleIndex,
-                            keyFields.length);
-                    int tupleLength = storedKeysAccessor1.getFieldLength(storedTuplePointer.tupleIndex,
-                            keyFields.length);
-                    aggregator.aggregate(accessor, tIndex, storedKeysAccessor1.getBuffer().array(), tupleOffset
-                            + storedKeysAccessor1.getFieldSlotsLength() + aggFieldOffset, tupleLength);
-                }
-                return true;
-            }
-
-            @Override
-            public List<ByteBuffer> getFrames() {
-                return frames;
-            }
-
-            @Override
-            public int getFrameCount() {
-                return dataFrameCount;
-            }
-
-            @Override
-            public void flushFrames(IFrameWriter writer, boolean isPartial) throws HyracksDataException {
-                FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-                writer.open();
-                appender.reset(outFrame, 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));
-                            // Reset the tuple for the partial result
-                            outputTupleBuilder.reset();
-                            for (int k = 0; k < keyFields.length; k++) {
-                                outputTupleBuilder.addField(storedKeysAccessor1, tIndex, k);
-                            }
-                            if (isPartial)
-                                aggregator.outputPartialResult(storedKeysAccessor1, tIndex, outputTupleBuilder);
-                            else
-                                aggregator.outputResult(storedKeysAccessor1, tIndex, outputTupleBuilder);
-                            while (!appender.append(outputTupleBuilder.getFieldEndOffsets(),
-                                    outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
-                                FrameUtils.flushFrame(outFrame, writer);
-                                appender.reset(outFrame, true);
-                            }
-                        } while (true);
-                    }
-                    if (appender.getTupleCount() != 0) {
-                        FrameUtils.flushFrame(outFrame, writer);
-                    }
-                    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
-                    ByteBuffer buffer = frames.get(frameIndex);
-                    storedKeysAccessor1.reset(buffer);
-
-                    outputTupleBuilder.reset();
-                    for (int k = 0; k < keyFields.length; k++) {
-                        outputTupleBuilder.addField(storedKeysAccessor1, tupleIndex, k);
-                    }
-                    if (isPartial)
-                        aggregator.outputPartialResult(storedKeysAccessor1, tupleIndex, outputTupleBuilder);
-                    else
-                        aggregator.outputResult(storedKeysAccessor1, tupleIndex, outputTupleBuilder);
-                    if (!appender.append(outputTupleBuilder.getFieldEndOffsets(), outputTupleBuilder.getByteArray(), 0,
-                            outputTupleBuilder.getSize())) {
-                        FrameUtils.flushFrame(outFrame, writer);
-                        appender.reset(outFrame, true);
-                        if (!appender.append(outputTupleBuilder.getFieldEndOffsets(),
-                                outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
-                            throw new IllegalStateException();
-                        }
-                    }
-                }
-                if (appender.getTupleCount() > 0) {
-                    FrameUtils.flushFrame(outFrame, writer);
-                }
-                aggregator.close();
-            }
-
-            /**
-             * 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.
-             */
-            private boolean nextAvailableFrame() {
-                // Return false if the number of frames is equal to the limit.
-                if (dataFrameCount + 1 >= framesLimit)
-                    return false;
-
-                if (frames.size() < framesLimit) {
-                    // Insert a new frame
-                    ByteBuffer frame = ctx.allocateFrame();
-                    frame.position(0);
-                    frame.limit(frame.capacity());
-                    frames.add(frame);
-                    appender.reset(frame, true);
-                    dataFrameCount = frames.size() - 1;
-                } else {
-                    // Reuse an old frame
-                    dataFrameCount++;
-                    ByteBuffer frame = frames.get(dataFrameCount);
-                    frame.position(0);
-                    frame.limit(frame.capacity());
-                    appender.reset(frame, true);
-                }
-                return true;
-            }
-
-            @Override
             public void sortFrames() {
                 int sfIdx = storedKeys[0];
                 int totalTCount = table.getTupleCount();
@@ -343,6 +191,224 @@
                 }
             }
 
+            @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));
+                    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<ByteBuffer> getFrames() {
+                return frames;
+            }
+
+            @Override
+            public int getFrameCount() {
+                return lastBufIndex;
+            }
+
+            @Override
+            public void flushFrames(IFrameWriter writer, boolean isPartial) throws HyracksDataException {
+                if (outputFrame == null) {
+                    outputFrame = ctx.allocateFrame();
+                }
+
+                if (outputAppender == null) {
+                    outputAppender = new FrameTupleAppender(outputFrame.capacity());
+                }
+
+                outputAppender.reset(outputFrame, true);
+
+                writer.open();
+
+                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));
+
+                            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())) {
+                                FrameUtils.flushFrame(outputFrame, writer);
+                                outputAppender.reset(outputFrame, 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);
+                    }
+                    if (outputAppender.getTupleCount() > 0) {
+                        FrameUtils.flushFrame(outputFrame, writer);
+                        outputAppender.reset(outputFrame, 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
+                    ByteBuffer buffer = frames.get(frameIndex);
+                    storedKeysAccessor1.reset(buffer);
+
+                    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())) {
+                        FrameUtils.flushFrame(outputFrame, writer);
+                        outputAppender.reset(outputFrame, 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.");
+                        }
+                    }
+                }
+                if (outputAppender.getTupleCount() > 0) {
+                    FrameUtils.flushFrame(outputFrame, writer);
+                    outputAppender.reset(outputFrame, true);
+                }
+                aggregator.close();
+            }
+
+            @Override
+            public void close() {
+                lastBufIndex = -1;
+                tPointers = null;
+                table.close();
+                frames.clear();
+                aggregateState.close();
+            }
+
+            /**
+             * 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.
+             */
+            private boolean nextAvailableFrame() {
+                // Return false if the number of frames is equal to the limit.
+                if (lastBufIndex + 1 >= framesLimit)
+                    return false;
+
+                if (frames.size() < framesLimit) {
+                    // Insert a new frame
+                    ByteBuffer frame = ctx.allocateFrame();
+                    frame.position(0);
+                    frame.limit(frame.capacity());
+                    frames.add(frame);
+                    stateAppender.reset(frame, true);
+                    lastBufIndex = frames.size() - 1;
+                } else {
+                    // Reuse an old frame
+                    lastBufIndex++;
+                    ByteBuffer frame = frames.get(lastBufIndex);
+                    frame.position(0);
+                    frame.limit(frame.capacity());
+                    stateAppender.reset(frame, true);
+                }
+                return true;
+            }
+
             private void sort(int[] tPointers, int offset, int length) {
                 int m = offset + (length >> 1);
                 int mTable = tPointers[m * 3];
@@ -437,14 +503,7 @@
                 }
             }
 
-            @Override
-            public void close() {
-                groupSize = 0;
-                dataFrameCount = -1;
-                tPointers = null;
-                table.close();
-                frames.clear();
-            }
         };
     }
+
 }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregator.java
deleted file mode 100644
index b0da898..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregator.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.group;
-
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-
-public interface IAccumulatingAggregator {
-    /**
-     * Called once per aggregator before calling accumulate for the first time.
-     * 
-     * @param accessor
-     *            - Accessor to the data tuple.
-     * @param tIndex
-     *            - Index of the tuple in the accessor.
-     * @throws HyracksDataException
-     */
-    public void init(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
-
-    /**
-     * Called once per tuple that belongs to this group.
-     * 
-     * @param accessor
-     *            - Accessor to data tuple.
-     * @param tIndex
-     *            - Index of tuple in the accessor.
-     * @throws HyracksDataException
-     */
-    public void accumulate(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
-
-    /**
-     * Called finally to emit output. This method is called until it returns true. The method is free to
-     * write out output to the provided appender until there is no more space and return false. It is the
-     * caller's responsibility to flush and make room in the appender before this method is called again.
-     * 
-     * @param appender
-     *            - Appender to write output to.
-     * @param accessor
-     *            - Accessor to access the key.
-     * @param tIndex
-     *            - Tuple index of the key in the accessor.
-     * @param keyFieldIndexes
-     *            - Field indexes of the key field.
-     * @return true if all output is written, false if the appender is full.
-     * @throws HyracksDataException
-     */
-    public boolean output(FrameTupleAppender appender, IFrameTupleAccessor accessor, int tIndex, int[] keyFieldIndexes)
-            throws HyracksDataException;
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptor.java
new file mode 100644
index 0000000..2cf978c
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptor.java
@@ -0,0 +1,107 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+/**
+ *
+ */
+public interface IAggregatorDescriptor {
+
+    /**
+     * Create an aggregate state
+     * 
+     * @return
+     */
+    public AggregateState createAggregateStates();
+
+    /**
+     * Initialize the state based on the input tuple.
+     * 
+     * @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)
+            throws HyracksDataException;
+
+    /**
+     * Reset the aggregator. The corresponding aggregate state should be reset
+     * 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();
+
+    /**
+     * Aggregate the value. Aggregate state should be updated correspondingly.
+     * 
+     * @param accessor
+     * @param tIndex
+     * @param data
+     *            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 state
+     *            The aggregate state.
+     * @throws HyracksDataException
+     */
+    public 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 state
+     *            The aggregation state.
+     * @throws HyracksDataException
+     */
+    public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            AggregateState state) throws HyracksDataException;
+
+    /**
+     * Output the final aggregation result.
+     * 
+     * @param fieldOutput
+     *            The data output for the output frame
+     * @param data
+     *            The buffer containing the aggregation state
+     * @param offset
+     * @param state
+     *            The aggregation state.
+     * @throws HyracksDataException
+     */
+    public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            AggregateState state) throws HyracksDataException;
+
+    public void close();
+
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptorFactory.java
similarity index 78%
rename from hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java
rename to hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptorFactory.java
index 7f6928b..9ff915d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptorFactory.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.hyracks.dataflow.std.aggregators;
+package edu.uci.ics.hyracks.dataflow.std.group;
 
 import java.io.Serializable;
 
@@ -20,18 +20,13 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
+/**
+ *
+ */
 public interface IAggregatorDescriptorFactory extends Serializable {
 
-    /**
-     * Create an aggregator.
-     * 
-     * @param ctx
-     * @param inRecordDescriptor
-     * @param outRecordDescriptor
-     * @param keyFields
-     * @return
-     * @throws HyracksDataException
-     */
     IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
-            RecordDescriptor outRecordDescriptor, int[] keyFields) throws HyracksDataException;
+            RecordDescriptor outRecordDescriptor, int[] keyFields, final int[] keyFieldsInPartialResults)
+            throws HyracksDataException;
+
 }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptor.java
new file mode 100644
index 0000000..a34fa73
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptor.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ *
+ */
+public interface IFieldAggregateDescriptor {
+
+    /**
+     * Initialize the state based on the input tuple.
+     * 
+     * @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.
+     * 
+     *            Note that we have an assumption that the initialization of the
+     *            binary state (if any) inserts the state fields into the buffer
+     *            in a appending fashion. This means that an arbitrary initial
+     *            size of the state can be accquired.
+     * @param state
+     *            The state to be initialized.
+     * @throws HyracksDataException
+     */
+    public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+            throws HyracksDataException;
+
+    /**
+     * Reset the aggregator. The corresponding aggregate state should be reset
+     * 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();
+
+    /**
+     * Aggregate the value. Aggregate state should be updated correspondingly.
+     * 
+     * @param accessor
+     * @param tIndex
+     * @param data
+     *            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.
+     * 
+     *            Here the length of binary state can be obtains from the state
+     *            parameter, and if the content to be filled into that is over-
+     *            flowing (larger than the reversed space), error should be
+     *            emit.
+     * @param offset
+     * @param state
+     *            The aggregate state.
+     * @throws HyracksDataException
+     */
+    public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset, 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 state
+     *            The aggregation state.
+     * @throws HyracksDataException
+     */
+    public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+            throws HyracksDataException;
+
+    /**
+     * Output the final aggregation result.
+     * 
+     * @param fieldOutput
+     *            The data output for the output frame
+     * @param data
+     *            The buffer containing the aggregation state
+     * @param offset
+     * @param state
+     *            The aggregation state.
+     * @throws HyracksDataException
+     */
+    public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+            throws HyracksDataException;
+
+    public boolean needsBinaryState();
+
+    public boolean needsObjectState();
+
+    public AggregateState createState();
+
+    /**
+     * Close the field aggregator
+     */
+    public void close();
+
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptorFactory.java
similarity index 82%
rename from hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
rename to hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptorFactory.java
index 978f671..6f50597 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptorFactory.java
@@ -20,7 +20,12 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
-public interface IAccumulatingAggregatorFactory extends Serializable {
-    IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
+/**
+ *
+ */
+public interface IFieldAggregateDescriptorFactory extends Serializable {
+
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
             RecordDescriptor outRecordDescriptor) throws HyracksDataException;
-}
\ No newline at end of file
+
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
index 7bd8cd8..ff80a23 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
@@ -21,9 +21,6 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
-/**
- * @author jarodwen
- */
 public interface ISpillableTable {
 
     public void close();
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
index 0ff1d1d..de9fac5 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
@@ -21,15 +21,11 @@
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory;
 
-/**
- * @author jarodwen
- */
 public interface ISpillableTableFactory extends Serializable {
     ISpillableTable buildSpillableTable(IHyracksTaskContext ctx, int[] keyFields,
             IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory normalizedKeyComputerFactory,
-            IAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor inRecordDescriptor,
+            IAggregatorDescriptorFactory aggregateFactory, RecordDescriptor inRecordDescriptor,
             RecordDescriptor outRecordDescriptor, int framesLimit) throws HyracksDataException;
 
 }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
index c678ac1..c718ee4 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
@@ -32,12 +32,12 @@
 public class PreclusteredGroupOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
     private final int[] groupFields;
     private final IBinaryComparatorFactory[] comparatorFactories;
-    private final IAccumulatingAggregatorFactory aggregatorFactory;
+    private final IAggregatorDescriptorFactory aggregatorFactory;
 
     private static final long serialVersionUID = 1L;
 
     public PreclusteredGroupOperatorDescriptor(JobSpecification spec, int[] groupFields,
-            IBinaryComparatorFactory[] comparatorFactories, IAccumulatingAggregatorFactory aggregatorFactory,
+            IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory aggregatorFactory,
             RecordDescriptor recordDescriptor) {
         super(spec, 1, 1);
         this.groupFields = groupFields;
@@ -47,15 +47,15 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            int partition, int nPartitions) throws HyracksDataException {
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
         for (int i = 0; i < comparatorFactories.length; ++i) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
         final RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
-        final IAccumulatingAggregator aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc,
-                recordDescriptors[0]);
+        final IAggregatorDescriptor aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc,
+                recordDescriptors[0], groupFields, groupFields);
         final ByteBuffer copyFrame = ctx.allocateFrame();
         final FrameTupleAccessor copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
         copyFrameAccessor.reset(copyFrame);
@@ -67,7 +67,8 @@
 
             @Override
             public void open() throws HyracksDataException {
-                pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregator, inRecordDesc, writer);
+                pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregator, inRecordDesc,
+                        recordDescriptors[0], writer);
                 pgw.open();
             }
 
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
index df7f9f9..89955c0 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
@@ -21,6 +21,7 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -28,28 +29,37 @@
 public class PreclusteredGroupWriter implements IFrameWriter {
     private final int[] groupFields;
     private final IBinaryComparator[] comparators;
-    private final IAccumulatingAggregator aggregator;
+    private final IAggregatorDescriptor aggregator;
+    private final AggregateState aggregateState;
     private final IFrameWriter writer;
     private final ByteBuffer copyFrame;
     private final FrameTupleAccessor inFrameAccessor;
     private final FrameTupleAccessor copyFrameAccessor;
+
     private final ByteBuffer outFrame;
     private final FrameTupleAppender appender;
+    private final ArrayTupleBuilder tupleBuilder;
+
     private boolean first;
 
     public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields, IBinaryComparator[] comparators,
-            IAccumulatingAggregator aggregator, RecordDescriptor inRecordDesc, IFrameWriter writer) {
+            IAggregatorDescriptor aggregator, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc,
+            IFrameWriter writer) {
         this.groupFields = groupFields;
         this.comparators = comparators;
         this.aggregator = aggregator;
+        this.aggregateState = aggregator.createAggregateStates();
         this.writer = writer;
         copyFrame = ctx.allocateFrame();
         inFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
         copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
         copyFrameAccessor.reset(copyFrame);
+
         outFrame = ctx.allocateFrame();
         appender = new FrameTupleAppender(ctx.getFrameSize());
         appender.reset(outFrame, true);
+
+        tupleBuilder = new ArrayTupleBuilder(outRecordDesc.getFields().length);
     }
 
     @Override
@@ -64,16 +74,23 @@
         int nTuples = inFrameAccessor.getTupleCount();
         for (int i = 0; i < nTuples; ++i) {
             if (first) {
-                aggregator.init(inFrameAccessor, i);
+
+                tupleBuilder.reset();
+                for (int j = 0; j < groupFields.length; j++) {
+                    tupleBuilder.addField(inFrameAccessor, i, groupFields[j]);
+                }
+                aggregator.init(tupleBuilder, inFrameAccessor, i, aggregateState);
+
                 first = false;
+
             } else {
                 if (i == 0) {
                     switchGroupIfRequired(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1, inFrameAccessor, i);
                 } else {
                     switchGroupIfRequired(inFrameAccessor, i - 1, inFrameAccessor, i);
                 }
+
             }
-            aggregator.accumulate(inFrameAccessor, i);
         }
         FrameUtils.copy(buffer, copyFrame);
     }
@@ -82,19 +99,36 @@
             FrameTupleAccessor currTupleAccessor, int currTupleIndex) throws HyracksDataException {
         if (!sameGroup(prevTupleAccessor, prevTupleIndex, currTupleAccessor, currTupleIndex)) {
             writeOutput(prevTupleAccessor, prevTupleIndex);
-            aggregator.init(currTupleAccessor, currTupleIndex);
+
+            tupleBuilder.reset();
+            for (int j = 0; j < groupFields.length; j++) {
+                tupleBuilder.addField(currTupleAccessor, currTupleIndex, groupFields[j]);
+            }
+            aggregator.init(tupleBuilder, currTupleAccessor, currTupleIndex, aggregateState);
+        } else {
+            aggregator.aggregate(currTupleAccessor, currTupleIndex, null, 0, aggregateState);
         }
     }
 
     private void writeOutput(final FrameTupleAccessor lastTupleAccessor, int lastTupleIndex)
             throws HyracksDataException {
-        if (!aggregator.output(appender, lastTupleAccessor, lastTupleIndex, groupFields)) {
-            FrameUtils.flushFrame(appender.getBuffer(), writer);
-            appender.reset(appender.getBuffer(), true);
-            if (!aggregator.output(appender, lastTupleAccessor, lastTupleIndex, groupFields)) {
-                throw new IllegalStateException();
+
+        tupleBuilder.reset();
+        for (int j = 0; j < groupFields.length; j++) {
+            tupleBuilder.addField(lastTupleAccessor, lastTupleIndex, groupFields[j]);
+        }
+        aggregator.outputFinalResult(tupleBuilder, lastTupleAccessor, lastTupleIndex, aggregateState);
+
+        if (!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+                tupleBuilder.getSize())) {
+            FrameUtils.flushFrame(outFrame, writer);
+            appender.reset(outFrame, true);
+            if (!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+                    tupleBuilder.getSize())) {
+                throw new HyracksDataException("The output cannot be fit into a frame.");
             }
         }
+
     }
 
     private boolean sameGroup(FrameTupleAccessor a1, int t1Idx, FrameTupleAccessor a2, int t2Idx) {
@@ -121,9 +155,10 @@
         if (!first) {
             writeOutput(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1);
             if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(appender.getBuffer(), writer);
+                FrameUtils.flushFrame(outFrame, writer);
             }
         }
+        aggregateState.close();
         writer.close();
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
new file mode 100644
index 0000000..2e781b5
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class AvgFieldGroupAggregatorFactory implements IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int aggField;
+
+    private final boolean useObjectState;
+
+    public AvgFieldGroupAggregatorFactory(int aggField, boolean useObjectState) {
+        this.aggField = aggField;
+        this.useObjectState = useObjectState;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.dataflow.std.aggregations.
+     * IFieldAggregateDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+
+        return new IFieldAggregateDescriptor() {
+
+            @Override
+            public void reset() {
+            }
+
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum, count;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                    count = IntegerSerializerDeserializer.getInt(data, offset + 4);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum = fields[0];
+                    count = fields[1];
+                }
+                try {
+                    fieldOutput.writeInt(sum);
+                    fieldOutput.writeInt(count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum, count;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                    count = IntegerSerializerDeserializer.getInt(data, offset + 4);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum = fields[0];
+                    count = fields[1];
+                }
+                try {
+                    fieldOutput.writeFloat((float) sum / count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+                int sum = 0;
+                int count = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+                count += 1;
+                if (!useObjectState) {
+                    try {
+                        fieldOutput.writeInt(sum);
+                        fieldOutput.writeInt(count);
+                    } catch (IOException e) {
+                        throw new HyracksDataException("I/O exception when initializing the aggregator.");
+                    }
+                } else {
+                    state.state = new Integer[] { sum, count };
+                }
+            }
+
+            @Override
+            public void close() {
+                // TODO Auto-generated method stub
+
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+                    AggregateState state) throws HyracksDataException {
+                int sum = 0, count = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+                count += 1;
+                if (!useObjectState) {
+                    ByteBuffer buf = ByteBuffer.wrap(data);
+                    sum += buf.getInt(offset);
+                    count += buf.getInt(offset + 4);
+                    buf.putInt(offset, sum);
+                    buf.putInt(offset + 4, count);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum += fields[0];
+                    count += fields[1];
+                    state.state = new Integer[] { sum, count };
+                }
+            }
+
+            @Override
+            public boolean needsObjectState() {
+                return useObjectState;
+            }
+
+            @Override
+            public boolean needsBinaryState() {
+                return !useObjectState;
+            }
+
+            @Override
+            public AggregateState createState() {
+                return new AggregateState(new Integer[] { 0, 0 });
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
new file mode 100644
index 0000000..cc5c1e1
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class AvgFieldMergeAggregatorFactory implements IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int aggField;
+
+    private final boolean useObjectState;
+
+    public AvgFieldMergeAggregatorFactory(int aggField, boolean useObjectState) {
+        this.aggField = aggField;
+        this.useObjectState = useObjectState;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.dataflow.std.aggregations.
+     * IFieldAggregateDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+
+        return new IFieldAggregateDescriptor() {
+
+            @Override
+            public void reset() {
+            }
+
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum, count;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                    count = IntegerSerializerDeserializer.getInt(data, offset + 4);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum = fields[0];
+                    count = fields[1];
+                }
+                try {
+                    fieldOutput.writeInt(sum);
+                    fieldOutput.writeInt(count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum, count;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                    count = IntegerSerializerDeserializer.getInt(data, offset + 4);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum = fields[0];
+                    count = fields[1];
+                }
+                try {
+                    fieldOutput.writeFloat((float) sum / count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void close() {
+                // TODO Auto-generated method stub
+
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+                    AggregateState state) throws HyracksDataException {
+                int sum = 0, count = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+                count += 1;
+                if (!useObjectState) {
+                    ByteBuffer buf = ByteBuffer.wrap(data);
+                    sum += buf.getInt(offset);
+                    count += buf.getInt(offset + 4);
+                    buf.putInt(offset, sum);
+                    buf.putInt(offset + 4, count);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum += fields[0];
+                    count += fields[1];
+                    state.state = new Integer[] { sum, count };
+                }
+            }
+
+            @Override
+            public boolean needsObjectState() {
+                return useObjectState;
+            }
+
+            @Override
+            public boolean needsBinaryState() {
+                return !useObjectState;
+            }
+
+            @Override
+            public AggregateState createState() {
+                return new AggregateState(new Integer[] { 0, 0 });
+            }
+
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+                int sum = 0;
+                int count = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+                count += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart + 4);
+                if (!useObjectState) {
+                    try {
+                        fieldOutput.writeInt(sum);
+                        fieldOutput.writeInt(count);
+                    } catch (IOException e) {
+                        throw new HyracksDataException("I/O exception when initializing the aggregator.");
+                    }
+                } else {
+                    state.state = new Integer[] { sum, count };
+                }
+            }
+        };
+    }
+
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
new file mode 100644
index 0000000..9bfec8e
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
@@ -0,0 +1,141 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class CountFieldAggregatorFactory implements IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final boolean useObjectState;
+
+    public CountFieldAggregatorFactory(boolean useObjectState) {
+        this.useObjectState = useObjectState;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.dataflow.std.aggregations.
+     * IFieldAggregateDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+        return new IFieldAggregateDescriptor() {
+
+            @Override
+            public void reset() {
+            }
+
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int count;
+                if (!useObjectState) {
+                    count = IntegerSerializerDeserializer.getInt(data, offset);
+                } else {
+                    count = (Integer) state.state;
+                }
+                try {
+                    fieldOutput.writeInt(count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int count;
+                if (!useObjectState) {
+                    count = IntegerSerializerDeserializer.getInt(data, offset);
+                } else {
+                    count = (Integer) state.state;
+                }
+                try {
+                    fieldOutput.writeInt(count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+                int count = 1;
+                if (!useObjectState) {
+                    try {
+                        fieldOutput.writeInt(count);
+                    } catch (IOException e) {
+                        throw new HyracksDataException("I/O exception when initializing the aggregator.");
+                    }
+                } else {
+                    state.state = count;
+                }
+            }
+
+            public boolean needsObjectState() {
+                return useObjectState;
+            }
+
+            public boolean needsBinaryState() {
+                return !useObjectState;
+            }
+
+            public AggregateState createState() {
+                return new AggregateState(new Integer(0));
+            }
+
+            @Override
+            public void close() {
+
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+                    AggregateState state) throws HyracksDataException {
+                int count = 1;
+                if (!useObjectState) {
+                    ByteBuffer buf = ByteBuffer.wrap(data);
+                    count += buf.getInt(offset);
+                    buf.putInt(offset, count);
+                } else {
+                    count += (Integer) state.state;
+                    state.state = count;
+                }
+            }
+        };
+    }
+
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
new file mode 100644
index 0000000..7d85deb
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
@@ -0,0 +1,157 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class IntSumFieldAggregatorFactory implements IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int aggField;
+
+    private final boolean useObjectState;
+
+    public IntSumFieldAggregatorFactory(int aggField, boolean useObjState) {
+        this.aggField = aggField;
+        this.useObjectState = useObjState;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.dataflow.std.aggregations.
+     * IFieldAggregateDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+
+        return new IFieldAggregateDescriptor() {
+
+            @Override
+            public void reset() {
+            }
+
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                } else {
+                    sum = (Integer) state.state;
+                }
+                try {
+                    fieldOutput.writeInt(sum);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                } else {
+                    sum = (Integer) state.state;
+                }
+                try {
+                    fieldOutput.writeInt(sum);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+
+                int sum = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+
+                if (!useObjectState) {
+                    try {
+                        fieldOutput.writeInt(sum);
+                    } catch (IOException e) {
+                        throw new HyracksDataException("I/O exception when initializing the aggregator.");
+                    }
+                } else {
+                    state.state = sum;
+                }
+            }
+
+            public boolean needsObjectState() {
+                return useObjectState;
+            }
+
+            public boolean needsBinaryState() {
+                return !useObjectState;
+            }
+
+            public AggregateState createState() {
+                return new AggregateState(new Integer(0));
+            }
+
+            @Override
+            public void close() {
+
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+                    AggregateState state) throws HyracksDataException {
+                int sum = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+
+                if (!useObjectState) {
+                    ByteBuffer buf = ByteBuffer.wrap(data);
+                    sum += buf.getInt(offset);
+                    buf.putInt(offset, sum);
+                } else {
+                    sum += (Integer) state.state;
+                    state.state = sum;
+                }
+            }
+        };
+    }
+
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MinMaxStringFieldAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MinMaxStringFieldAggregatorFactory.java
new file mode 100644
index 0000000..94ebcbd
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MinMaxStringFieldAggregatorFactory.java
@@ -0,0 +1,199 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class MinMaxStringFieldAggregatorFactory implements IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int aggField;
+
+    private final boolean isMax;
+
+    private final boolean hasBinaryState;
+
+    public MinMaxStringFieldAggregatorFactory(int aggField, boolean isMax, boolean hasBinaryState) {
+        this.aggField = aggField;
+        this.isMax = isMax;
+        this.hasBinaryState = hasBinaryState;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int[])
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+        return new IFieldAggregateDescriptor() {
+
+            @Override
+            public void reset() {
+            }
+
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                try {
+                    if (hasBinaryState) {
+                        int stateIdx = IntegerSerializerDeserializer.getInt(data, offset);
+                        Object[] storedState = (Object[]) state.state;
+                        fieldOutput.writeUTF((String) storedState[stateIdx]);
+                    } else {
+                        fieldOutput.writeUTF((String) state.state);
+                    }
+                } catch (IOException e) {
+                    throw new HyracksDataException(
+                            "I/O exception when writing a string to the output writer in MinMaxStringAggregatorFactory.");
+                }
+            }
+
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                try {
+                    if (hasBinaryState) {
+                        int stateIdx = IntegerSerializerDeserializer.getInt(data, offset);
+                        Object[] storedState = (Object[]) state.state;
+                        fieldOutput.writeUTF((String) storedState[stateIdx]);
+                    } else {
+                        fieldOutput.writeUTF((String) state.state);
+                    }
+                } catch (IOException e) {
+                    throw new HyracksDataException(
+                            "I/O exception when writing a string to the output writer in MinMaxStringAggregatorFactory.");
+                }
+            }
+
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                int fieldLength = accessor.getFieldLength(tIndex, aggField);
+                String strField = UTF8StringSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
+                        new ByteArrayInputStream(accessor.getBuffer().array(), tupleOffset
+                                + accessor.getFieldSlotsLength() + fieldStart, fieldLength)));
+                if (hasBinaryState) {
+                    // Object-binary-state
+                    Object[] storedState;
+                    if (state.state == null) {
+                        storedState = new Object[8];
+                        storedState[0] = new Integer(0);
+                        state.state = storedState;
+                    } else {
+                        storedState = (Object[]) state.state;
+                    }
+                    int stateCount = (Integer) (storedState[0]);
+                    if (stateCount + 1 >= storedState.length) {
+                        storedState = Arrays.copyOf(storedState, storedState.length * 2);
+                        state.state = storedState;
+                    }
+
+                    stateCount++;
+                    storedState[0] = stateCount;
+                    storedState[stateCount] = strField;
+                    try {
+                        fieldOutput.writeInt(stateCount);
+                    } catch (IOException e) {
+                        throw new HyracksDataException(e.fillInStackTrace());
+                    }
+                } else {
+                    // Only object-state
+                    state.state = strField;
+                }
+            }
+
+            @Override
+            public void close() {
+                // TODO Auto-generated method stub
+
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+                    AggregateState state) throws HyracksDataException {
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                int fieldLength = accessor.getFieldLength(tIndex, aggField);
+                String strField = UTF8StringSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
+                        new ByteArrayInputStream(accessor.getBuffer().array(), tupleOffset
+                                + accessor.getFieldSlotsLength() + fieldStart, fieldLength)));
+                if (hasBinaryState) {
+                    int stateIdx = IntegerSerializerDeserializer.getInt(data, offset);
+
+                    Object[] storedState = (Object[]) state.state;
+
+                    if (isMax) {
+                        if (strField.length() > ((String) (storedState[stateIdx])).length()) {
+                            storedState[stateIdx] = strField;
+                        }
+                    } else {
+                        if (strField.length() < ((String) (storedState[stateIdx])).length()) {
+                            storedState[stateIdx] = strField;
+                        }
+                    }
+                } else {
+                    if (isMax) {
+                        if (strField.length() > ((String) (state.state)).length()) {
+                            state.state = strField;
+                        }
+                    } else {
+                        if (strField.length() < ((String) (state.state)).length()) {
+                            state.state = strField;
+                        }
+                    }
+                }
+            }
+
+            public boolean needsObjectState() {
+                return true;
+            }
+
+            public boolean needsBinaryState() {
+                return hasBinaryState;
+            }
+
+            public AggregateState createState() {
+                return new AggregateState();
+            }
+
+        };
+    }
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java
new file mode 100644
index 0000000..6c436a9
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java
@@ -0,0 +1,170 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class MultiFieldsAggregatorFactory implements IAggregatorDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final IFieldAggregateDescriptorFactory[] aggregatorFactories;
+    private int[] keys;
+
+    public MultiFieldsAggregatorFactory(int[] keys, IFieldAggregateDescriptorFactory[] aggregatorFactories) {
+        this.keys = keys;
+        this.aggregatorFactories = aggregatorFactories;
+    }
+
+    public MultiFieldsAggregatorFactory(IFieldAggregateDescriptorFactory[] aggregatorFactories) {
+        this.aggregatorFactories = aggregatorFactories;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.dataflow.std.aggregations.IAggregatorDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor, final int[] keyFields, final int[] keyFieldsInPartialResults)
+            throws HyracksDataException {
+
+        final IFieldAggregateDescriptor[] aggregators = new IFieldAggregateDescriptor[aggregatorFactories.length];
+        for (int i = 0; i < aggregators.length; i++) {
+            aggregators[i] = aggregatorFactories[i].createAggregator(ctx, inRecordDescriptor, outRecordDescriptor);
+        }
+
+        if (this.keys == null) {
+            this.keys = keyFields;
+        }
+
+        return new IAggregatorDescriptor() {
+
+            @Override
+            public void reset() {
+                for (int i = 0; i < aggregators.length; i++) {
+                    aggregators[i].reset();
+                }
+            }
+
+            @Override
+            public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                DataOutput dos = tupleBuilder.getDataOutput();
+
+                int tupleOffset = accessor.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,
+                            ((AggregateState[]) state.state)[i]);
+                    tupleBuilder.addFieldEndOffset();
+                }
+
+            }
+
+            @Override
+            public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                DataOutput dos = tupleBuilder.getDataOutput();
+
+                int tupleOffset = accessor.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,
+                                ((AggregateState[]) state.state)[i]);
+                    } else {
+                        aggregators[i].outputFinalResult(dos, null, 0, ((AggregateState[]) state.state)[i]);
+                    }
+                    tupleBuilder.addFieldEndOffset();
+                }
+            }
+
+            @Override
+            public void init(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                DataOutput dos = tupleBuilder.getDataOutput();
+
+                for (int i = 0; i < aggregators.length; i++) {
+                    aggregators[i].init(accessor, tIndex, dos, ((AggregateState[]) state.state)[i]);
+                    if (aggregators[i].needsBinaryState()) {
+                        tupleBuilder.addFieldEndOffset();
+                    }
+                }
+            }
+
+            @Override
+            public AggregateState createAggregateStates() {
+                AggregateState[] states = new AggregateState[aggregators.length];
+                for (int i = 0; i < states.length; i++) {
+                    states[i] = aggregators[i].createState();
+                }
+                return new AggregateState(states);
+            }
+
+            @Override
+            public void close() {
+                for (int i = 0; i < aggregators.length; i++) {
+                    aggregators[i].close();
+                }
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor,
+                    int stateTupleIndex, AggregateState state) throws HyracksDataException {
+                if (stateAccessor != null) {
+                    int stateTupleOffset = stateAccessor.getTupleStartOffset(stateTupleIndex);
+                    int fieldIndex = 0;
+                    for (int i = 0; i < aggregators.length; i++) {
+                        if (aggregators[i].needsBinaryState()) {
+                            int stateFieldOffset = stateAccessor.getFieldStartOffset(stateTupleIndex, keys.length
+                                    + fieldIndex);
+                            aggregators[i].aggregate(accessor, tIndex, stateAccessor.getBuffer().array(),
+                                    stateTupleOffset + stateAccessor.getFieldSlotsLength() + stateFieldOffset,
+                                    ((AggregateState[]) state.state)[i]);
+                            fieldIndex++;
+                        } else {
+                            aggregators[i].aggregate(accessor, tIndex, null, 0, ((AggregateState[]) state.state)[i]);
+                        }
+                    }
+                } else {
+                    for (int i = 0; i < aggregators.length; i++) {
+                        aggregators[i].aggregate(accessor, tIndex, null, 0, ((AggregateState[]) state.state)[i]);
+                    }
+                }
+            }
+        };
+    }
+}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
index eb58cb9..84caadf 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
@@ -72,8 +72,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            int partition, int nPartitions) {
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new MapperOperator(),
                 recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
     }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
index b0e3dbe..85a6912 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
@@ -40,8 +40,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            int partition, int nPartitions) throws HyracksDataException {
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
         return new ConstantTupleSourceOperatorNodePushable(ctx, fieldSlots, tupleData, tupleSize);
     }
 }
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
index b6a96e3..8e0ac79 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
@@ -32,8 +32,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            int partition, int nPartitions) {
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new AbstractUnaryInputSinkOperatorNodePushable() {
             @Override
             public void open() throws HyracksDataException {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
index 2d4a4d0..7effbb0 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
@@ -61,8 +61,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-            int partition, int nPartitions) {
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
         return new DeserializedOperatorNodePushable(ctx, new PrinterOperator(),
                 recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
     }
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
index 5d62607..6874f77 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
@@ -24,8 +24,8 @@
     }
 
     @Override
-    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IRecordDescriptorProvider recordDescProvider,
-            int partition, int nPartitions)
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
             throws HyracksDataException {
         return new AbstractUnaryInputOperatorNodePushable() {
             private final IFrameWriter[] writers = new IFrameWriter[outputArity];
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
index e7fb9ac..3e99000 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
@@ -72,8 +72,8 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-                final int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
             IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
                 private CollectTaskState state;
 
@@ -117,8 +117,8 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-                final int partition, int nPartitions) {
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
             IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
                 private IOpenableDataWriter<Object[]> writer;
 
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
index fcf4978..f55eb3d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
@@ -55,8 +55,8 @@
         }
 
         @Override
-        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
-                int partition, int nPartitions)
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
                 throws HyracksDataException {
             RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
             return new UnionOperator(ctx, inRecordDesc);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
new file mode 100644
index 0000000..2813a62
--- /dev/null
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
@@ -0,0 +1,908 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.tests.integration;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.HashGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableTableFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.AvgFieldGroupAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.AvgFieldMergeAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MinMaxStringFieldAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
+
+/**
+ *
+ */
+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 RecordDescriptor desc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+            FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+    final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
+            UTF8StringParserFactory.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, }, '|');
+
+    private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, String prefix)
+            throws IOException {
+
+        AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec,
+                new ConstantFileSplitProvider(new FileSplit[] {
+                        new FileSplit(NC1_ID, createTempFile().getAbsolutePath()),
+                        new FileSplit(NC2_ID, createTempFile().getAbsolutePath()) }), "\t");
+
+        return printer;
+    }
+
+    @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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.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) }),
+                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();
+
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
+
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE });
+
+        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 IntSumFieldAggregatorFactory(3, true) }),
+                outputRec);
+
+        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 singleKeySumExtGroupTest() 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE });
+
+        int[] keyFields = new int[] { 0 };
+        int frameLimits = 4;
+        int tableSize = 8;
+
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new IntSumFieldAggregatorFactory(3, false) }), new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+                                new IntSumFieldAggregatorFactory(2, false) }), 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, "singleKeySumExtGroupTest");
+
+        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 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, 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();
+
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
+
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
+
+        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 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) }));
+        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 singleKeyAvgExtGroupTest() 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
+
+        int[] keyFields = new int[] { 0 };
+        int frameLimits = 4;
+        int tableSize = 8;
+
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, 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 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);
+
+        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 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.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 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();
+
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
+
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
+
+        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);
+
+        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 singleKeyMinMaxStringExtGroupTest() 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                UTF8StringSerializerDeserializer.INSTANCE });
+
+        int[] keyFields = new int[] { 0 };
+        int frameLimits = 4;
+        int tableSize = 8;
+
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                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);
+
+        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");
+
+        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 multiKeySumPreClusterGroupTest() 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.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 IntSumFieldAggregatorFactory(3, 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) }));
+        spec.connect(conn1, csvScanner, 0, grouper, 0);
+
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
+
+        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 multiKeySumExtGroupTest() 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+        int[] keyFields = new int[] { 8, 0 };
+        int frameLimits = 4;
+        int tableSize = 8;
+
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, 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);
+
+        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, "multiKeySumExtGroupTest");
+
+        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 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                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();
+
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
+
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                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);
+
+        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 multiKeyAvgExtGroupTest() 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+                FloatSerializerDeserializer.INSTANCE });
+
+        int[] keyFields = new int[] { 8, 0 };
+        int frameLimits = 4;
+        int tableSize = 8;
+
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, 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(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);
+
+        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, "multiKeyAvgExtGroupTest");
+
+        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 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.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 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();
+
+        FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+                desc);
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
+
+        RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.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 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) }));
+        spec.connect(conn1, csvScanner, 0, grouper, 0);
+
+        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringPreClusterGroupTest");
+
+        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 multiKeyMinMaxStringExtGroupTest() 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[] {
+                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+                IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+        int[] keyFields = new int[] { 8, 0 };
+        int frameLimits = 4;
+        int tableSize = 8;
+
+        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                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);
+
+        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, "multiKeyMinMaxStringExtGroupTest");
+
+        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);
+    }
+
+}
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index c43238d..ce72ec5 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -36,9 +36,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IFieldValueResultingAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
@@ -48,7 +45,10 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
 
@@ -80,7 +80,7 @@
                 spec,
                 new int[] { 0 },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                 desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID);
 
@@ -90,12 +90,9 @@
 
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                new int[] { 1 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
-                desc3);
+        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -155,7 +152,7 @@
                 spec,
                 new int[] { 0 },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                 desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
@@ -165,12 +162,9 @@
 
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                new int[] { 1 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
-                desc3);
+        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
@@ -230,7 +224,7 @@
                 spec,
                 new int[] { 0 },
                 new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                 desc2);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
 
@@ -240,12 +234,9 @@
 
         RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(
-                spec,
-                new int[] { 1 },
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                new MultiAggregatorFactory(new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
-                desc3);
+        PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+                        new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
 
         IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java
deleted file mode 100644
index be547e7..0000000
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java
+++ /dev/null
@@ -1,382 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.tests.spillable;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.AvgAggregatorDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.ConcatAggregatorDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IntSumAggregatorDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableGroupingTableFactory;
-import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
-import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
-
-/**
- * 
- */
-public class ExternalAggregateTest extends AbstractIntegrationTest {
-
-    final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC2_ID,
-            new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
-
-    static final boolean isOutputFile = true;
-
-    final RecordDescriptor desc = new RecordDescriptor(new ISerializerDeserializer[] {
-            UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
-            IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
-            IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
-            FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-    final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
-            UTF8StringParserFactory.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, }, '|');
-
-    private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, boolean isFile, String prefix)
-            throws IOException {
-        AbstractSingleActivityOperatorDescriptor printer;
-
-        if (!isOutputFile)
-            printer = new PrinterOperatorDescriptor(spec);
-        else
-            printer = new PlainFileWriterOperatorDescriptor(spec, new ConstantFileSplitProvider(new FileSplit[] {
-                    new FileSplit(NC1_ID, createTempFile().getAbsolutePath()),
-                    new FileSplit(NC2_ID, createTempFile().getAbsolutePath()) }), "\t");
-
-        return printer;
-    }
-
-    @Test
-    public void hashSingleKeyScalarGroupTest() 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[] {
-                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 0 };
-        int frameLimits = 3;
-        int tableSize = 8;
-
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(), new CountAggregatorDescriptorFactory(),
-                new IntSumAggregatorDescriptorFactory(keyFields.length), outputRec,
-                new HashSpillableGroupingTableFactory(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, isOutputFile,
-                "hashSingleKeyScalarGroupTest");
-
-        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 hashMultipleKeyScalarGroupTest() 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[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE, });
-
-        int[] keyFields = new int[] { 0, 9 };
-        int frameLimits = 3;
-        int tableSize = 8;
-
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(), new IntSumAggregatorDescriptorFactory(1),
-                new IntSumAggregatorDescriptorFactory(keyFields.length), outputRec,
-                new HashSpillableGroupingTableFactory(new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                                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),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
-        spec.connect(conn1, csvScanner, 0, grouper, 0);
-
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, isOutputFile,
-                "hashMultipleKeyScalarGroupTest");
-        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 hashMultipleKeyMultipleScalarGroupTest() 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[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE, });
-
-        int[] keyFields = new int[] { 0, 9 };
-        int frameLimits = 3;
-        int tableSize = 8;
-
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(), new MultiAggregatorDescriptorFactory(
-                        new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(1, 2),
-                                new IntSumAggregatorDescriptorFactory(2, 3) }), new MultiAggregatorDescriptorFactory(
-                        new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(2, 2),
-                                new IntSumAggregatorDescriptorFactory(3, 3) }), outputRec,
-                new HashSpillableGroupingTableFactory(new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                                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),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
-        spec.connect(conn1, csvScanner, 0, grouper, 0);
-
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, isOutputFile,
-                "hashMultipleKeyMultipleScalarGroupTest");
-        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 hashMultipleKeyNonScalarGroupTest() 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[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 0 };
-        int frameLimits = 3;
-        int tableSize = 8;
-
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(), new ConcatAggregatorDescriptorFactory(9),
-                new ConcatAggregatorDescriptorFactory(keyFields.length), outputRec,
-                new HashSpillableGroupingTableFactory(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, isOutputFile,
-                "hashMultipleKeyNonScalarGroupTest");
-        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 hashMultipleKeyMultipleFieldsGroupTest() 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[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 0, 9 };
-        int frameLimits = 3;
-        int tableSize = 8;
-
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(), new MultiAggregatorDescriptorFactory(
-                        new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(1, 2),
-                                new IntSumAggregatorDescriptorFactory(2, 3),
-                                new ConcatAggregatorDescriptorFactory(9, 4) }), new MultiAggregatorDescriptorFactory(
-                        new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(2, 2),
-                                new IntSumAggregatorDescriptorFactory(3, 3),
-                                new ConcatAggregatorDescriptorFactory(4, 4) }), outputRec,
-                new HashSpillableGroupingTableFactory(new FieldHashPartitionComputerFactory(keyFields,
-                        new IBinaryHashFunctionFactory[] {
-                                PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
-                                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),
-                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
-        spec.connect(conn1, csvScanner, 0, grouper, 0);
-
-        AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, isOutputFile,
-                "hashMultipleKeyMultipleFieldsGroupTest");
-        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 hashSingleKeyScalarAvgGroupTest() 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[] {
-                UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-
-        int[] keyFields = new int[] { 0 };
-        int frameLimits = 3;
-        int tableSize = 8;
-
-        ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
-                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                new UTF8StringNormalizedKeyComputerFactory(), new AvgAggregatorDescriptorFactory(1),
-                new AvgAggregatorDescriptorFactory(keyFields.length), outputRec, new HashSpillableGroupingTableFactory(
-                        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, isOutputFile,
-                "hashSingleKeyScalarGroupTest");
-
-        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);
-    }
-}
diff --git a/hyracks-examples/text-example/textclient/pom.xml b/hyracks-examples/text-example/textclient/pom.xml
index 2642d31..c6f3832 100644
--- a/hyracks-examples/text-example/textclient/pom.xml
+++ b/hyracks-examples/text-example/textclient/pom.xml
@@ -41,6 +41,7 @@
         <artifactId>appassembler-maven-plugin</artifactId>
         <executions>
           <execution>
+          <id>textclient</id>
             <configuration>
               <programs>
                 <program>
@@ -56,6 +57,23 @@
               <goal>assemble</goal>
             </goals>
           </execution>
+          <execution>
+          	<id>groupclient</id>
+            <configuration>
+              <programs>
+                <program>
+                  <mainClass>edu.uci.ics.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>
       </plugin>
       <plugin>
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
index f594d7e..66f7efc 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
@@ -43,13 +43,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IFieldValueResultingAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IntSumAggregatorDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorDescriptorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
@@ -61,10 +54,12 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FrameFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.HashGroupOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableGroupingTableFactory;
-import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableTableFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 
 /**
@@ -102,6 +97,9 @@
 
         @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;
     }
 
     /**
@@ -116,10 +114,11 @@
 
         JobSpecification job;
 
-        for (int i = 0; i < 3; i++) {
+        for (int i = 0; i < 6; i++) {
             long start = System.currentTimeMillis();
-            job = createJob(parseFileSplits(options.inFileSplits), parseFileSplits(options.outFileSplits, i % 2),
-                    options.htSize, options.sbSize, options.framesLimit, options.sortOutput, i % 3, options.outPlain);
+            job = createJob(parseFileSplits(options.inFileSplits), parseFileSplits(options.outFileSplits, i),
+                    options.htSize, options.sbSize, options.framesLimit, options.sortOutput, options.algo,
+                    options.outPlain);
 
             System.out.print(i + "\t" + (System.currentTimeMillis() - start));
             start = System.currentTimeMillis();
@@ -201,102 +200,100 @@
         AbstractOperatorDescriptor grouper;
 
         switch (alg) {
-            case 0: // External hash group
-                grouper = new ExternalGroupOperatorDescriptor(
-                        spec,
-                        keys,
-                        framesLimit,
-                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                        new IntegerNormalizedKeyComputerFactory(),
-                        new MultiAggregatorDescriptorFactory(
-                                new IAggregatorDescriptorFactory[] { new CountAggregatorDescriptorFactory() }),
-                        new MultiAggregatorDescriptorFactory(
-                                new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(keys.length) }),
-                        outDesc, new HashSpillableGroupingTableFactory(new FieldHashPartitionComputerFactory(keys,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(IntegerPointable.FACTORY) }), htSize), false);
+            case 0: // new external hash graph
+                grouper = new edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor(spec, keys,
+                        framesLimit, 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 scanGroupConn = new MToNPartitioningConnectorDescriptor(spec,
-                        new FieldHashPartitionComputerFactory(keys,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(IntegerPointable.FACTORY) }));
-                spec.connect(scanGroupConn, fileScanner, 0, grouper, 0);
+                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 + pre-cluster
-                ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(
-                        spec,
-                        framesLimit,
-                        keys,
-                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                        inDesc);
-                createPartitionConstraint(spec, sorter, inSplits);
+            case 1: // External-sort + new-precluster
+                ExternalSortOperatorDescriptor sorter2 = new ExternalSortOperatorDescriptor(spec, framesLimit, keys,
+                        new IBinaryComparatorFactory[] {
+                        // PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, inDesc);
+                createPartitionConstraint(spec, sorter2, inSplits);
 
                 // Connect scan operator with the sorter
-                IConnectorDescriptor scanSortConn = new MToNPartitioningConnectorDescriptor(spec,
-                        new FieldHashPartitionComputerFactory(keys,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(IntegerPointable.FACTORY) }));
-                spec.connect(scanSortConn, fileScanner, 0, sorter, 0);
+                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 PreclusteredGroupOperatorDescriptor(
-                        spec,
-                        keys,
-                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                        new MultiAggregatorFactory(
-                                new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc);
+                grouper = new edu.uci.ics.hyracks.dataflow.std.group.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 sortGroupConn = new OneToOneConnectorDescriptor(spec);
-                spec.connect(sortGroupConn, sorter, 0, grouper, 0);
+                OneToOneConnectorDescriptor sortGroupConn2 = new OneToOneConnectorDescriptor(spec);
+                spec.connect(sortGroupConn2, sorter2, 0, grouper, 0);
                 break;
-            case 2: // In-memory hash group
-                grouper = new HashGroupOperatorDescriptor(
-                        spec,
-                        keys,
-                        new FieldHashPartitionComputerFactory(keys,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(IntegerPointable.FACTORY) }),
-                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                        new MultiAggregatorFactory(
-                                new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }), outDesc,
-                        htSize);
+            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 scanConn = new MToNPartitioningConnectorDescriptor(spec,
-                        new FieldHashPartitionComputerFactory(keys,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(IntegerPointable.FACTORY) }));
-                spec.connect(scanConn, fileScanner, 0, grouper, 0);
+                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 ExternalGroupOperatorDescriptor(
-                        spec,
-                        keys,
-                        framesLimit,
-                        new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
-                        new IntegerNormalizedKeyComputerFactory(),
-                        new MultiAggregatorDescriptorFactory(
-                                new IAggregatorDescriptorFactory[] { new CountAggregatorDescriptorFactory() }),
-                        new MultiAggregatorDescriptorFactory(
-                                new IAggregatorDescriptorFactory[] { new IntSumAggregatorDescriptorFactory(keys.length) }),
-                        outDesc, new HashSpillableGroupingTableFactory(new FieldHashPartitionComputerFactory(keys,
-                                new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                        .of(IntegerPointable.FACTORY) }), htSize), false);
+                grouper = new edu.uci.ics.hyracks.dataflow.std.group.ExternalGroupOperatorDescriptor(spec, keys,
+                        framesLimit, 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) }));
+                        new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
+                        // PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
                 spec.connect(scanGroupConnDef, fileScanner, 0, grouper, 0);
         }
 
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
index fe3cc03..1708259 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
@@ -40,9 +40,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IFieldValueResultingAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
@@ -52,7 +49,10 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.group.HashGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
 import edu.uci.ics.hyracks.examples.text.WordTupleParserFactory;
@@ -148,8 +148,8 @@
                             new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                     .of(UTF8StringPointable.FACTORY) }),
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    new MultiAggregatorFactory(
-                            new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                    new MultiFieldsAggregatorFactory(
+                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                     groupResultDesc, htSize);
             createPartitionConstraint(spec, gBy, outSplits);
             IConnectorDescriptor scanGroupConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -176,8 +176,8 @@
                     spec,
                     keys,
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    new MultiAggregatorFactory(
-                            new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                    new MultiFieldsAggregatorFactory(
+                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                     groupResultDesc);
             createPartitionConstraint(spec, gBy, outSplits);
             OneToOneConnectorDescriptor sortGroupConn = new OneToOneConnectorDescriptor(spec);
diff --git a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index e876f65..72533e7 100644
--- a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -46,9 +46,6 @@
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.IFieldValueResultingAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
@@ -58,6 +55,9 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FrameFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.group.HashGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
 import edu.uci.ics.hyracks.dataflow.std.join.GraceHashJoinOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.HybridHashJoinOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
@@ -268,8 +268,8 @@
                             new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
                                     .of(UTF8StringPointable.FACTORY) }),
                     new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    new MultiAggregatorFactory(
-                            new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
+                    new MultiFieldsAggregatorFactory(
+                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
                     groupResultDesc, 16);
             createPartitionConstraint(spec, gby, resultSplits);