You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by ji...@apache.org on 2015/06/18 06:22:18 UTC

[01/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Repository: incubator-asterixdb-hyracks
Updated Branches:
  refs/heads/master 0e5d5315e -> 0d87a57f7


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index 8598b70..231adbd 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -15,11 +15,10 @@
 
 package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow;
 
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 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;
@@ -27,9 +26,7 @@ import edu.uci.ics.hyracks.data.std.util.GrowableArray;
 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.ByteBufferInputStream;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
@@ -50,7 +47,6 @@ public class BinaryTokenizerOperatorNodePushable extends
     private ArrayTupleBuilder builder;
     private GrowableArray builderData;
     private FrameTupleAppender appender;
-    private ByteBuffer writeBuffer;
 
     public BinaryTokenizerOperatorNodePushable(IHyracksTaskContext ctx,
             RecordDescriptor inputRecDesc, RecordDescriptor outputRecDesc,
@@ -68,12 +64,10 @@ public class BinaryTokenizerOperatorNodePushable extends
 
     @Override
     public void open() throws HyracksDataException {
-        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
-        writeBuffer = ctx.allocateFrame();
+        accessor = new FrameTupleAccessor(inputRecDesc);
         builder = new ArrayTupleBuilder(outputRecDesc.getFieldCount());
         builderData = builder.getFieldData();
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        appender.reset(writeBuffer, true);
+        appender = new FrameTupleAppender(new VSizeFrame(ctx), true);
         writer.open();
     }
 
@@ -157,19 +151,8 @@ public class BinaryTokenizerOperatorNodePushable extends
 
                 }
 
-                if (!appender.append(builder.getFieldEndOffsets(),
-                        builder.getByteArray(), 0, builder.getSize())) {
-                    FrameUtils.flushFrame(writeBuffer, writer);
-                    appender.reset(writeBuffer, true);
-
-                    if (!appender.append(builder.getFieldEndOffsets(),
-                            builder.getByteArray(), 0, builder.getSize())) {
-                        throw new HyracksDataException("Record size ("
-                                + builder.getSize()
-                                + ") larger than frame size ("
-                                + appender.getBuffer().capacity() + ")");
-                    }
-                }
+                FrameUtils.appendToWriter(writer, appender, builder.getFieldEndOffsets(),
+                        builder.getByteArray(), 0, builder.getSize());
 
             }
 
@@ -179,9 +162,7 @@ public class BinaryTokenizerOperatorNodePushable extends
 
     @Override
     public void close() throws HyracksDataException {
-        if (appender.getTupleCount() > 0) {
-            FrameUtils.flushFrame(writeBuffer, writer);
-        }
+        appender.flush(writer, true);
         writer.close();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
index ddaeac0..95ec645 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
@@ -15,8 +15,10 @@
 
 package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java
index 59690c3..3c6c4cd 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java
@@ -67,6 +67,11 @@ public class FixedSizeFrameTupleAccessor implements IFrameTupleAccessor {
     }
 
     @Override
+    public int getTupleLength(int tupleIndex) {
+        return getTupleEndOffset(tupleIndex) - getTupleStartOffset(tupleIndex);
+    }
+
+    @Override
     public int getFieldSlotsLength() {
         return 0;
     }
@@ -92,6 +97,11 @@ public class FixedSizeFrameTupleAccessor implements IFrameTupleAccessor {
     }
 
     @Override
+    public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+        return getTupleStartOffset(tupleIndex) + getFieldSlotsLength() + getFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
     public void reset(ByteBuffer buffer) {
         this.buffer = buffer;
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 1aded5f..a479815 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -549,7 +549,7 @@ public class OnDiskInvertedIndex implements IInvertedIndex {
         private final int FRAME_SIZE = 32768;
 
         @Override
-        public int getFrameSize() {
+        public int getInitialFrameSize() {
             return FRAME_SIZE;
         }
 
@@ -563,8 +563,16 @@ public class OnDiskInvertedIndex implements IInvertedIndex {
             return ByteBuffer.allocate(FRAME_SIZE);
         }
 
+        @Override public ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+            return ByteBuffer.allocate(bytes);
+        }
+
+        @Override public ByteBuffer reallocateFrame(ByteBuffer bytes, int newSizeInBytes, boolean copyOldData) throws HyracksDataException {
+            throw new HyracksDataException("TODO");
+        }
+
         @Override
-        public void deallocateFrames(int frameCount) {
+        public void deallocateFrames(int bytes) {
             // TODO Auto-generated method stub
 
         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
index f3b019f..5b68c7b 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -19,18 +19,18 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 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.FrameTupleAppenderAccessor;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-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.storage.am.common.ophelpers.MultiComparator;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
@@ -59,9 +59,8 @@ public abstract class AbstractTOccurrenceSearcher implements IInvertedIndexSearc
     protected final MultiComparator invListCmp;
 
     protected final ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(QUERY_TOKEN_REC_DESC.getFieldCount());
-    protected final ByteBuffer queryTokenFrame;
-    protected final FrameTupleAppender queryTokenAppender;
-    protected final FrameTupleAccessor queryTokenAccessor;
+    protected final IFrame queryTokenFrame;
+    protected final FrameTupleAppenderAccessor queryTokenAppender;
     protected final FrameTupleReference searchKey = new FrameTupleReference();
 
     protected int occurrenceThreshold;
@@ -78,10 +77,9 @@ public abstract class AbstractTOccurrenceSearcher implements IInvertedIndexSearc
         this.invListCursorFactory = new InvertedListCursorFactory(invIndex);
         this.invListCursorCache = new ObjectCache<IInvertedListCursor>(invListCursorFactory, OBJECT_CACHE_INIT_SIZE,
                 OBJECT_CACHE_EXPAND_SIZE);
-        this.queryTokenFrame = ctx.allocateFrame();
-        this.queryTokenAppender = new FrameTupleAppender(ctx.getFrameSize());
-        this.queryTokenAccessor = new FrameTupleAccessor(ctx.getFrameSize(), QUERY_TOKEN_REC_DESC);
-        this.queryTokenAccessor.reset(queryTokenFrame);
+        this.queryTokenFrame =  new VSizeFrame(ctx);
+        this.queryTokenAppender = new FrameTupleAppenderAccessor(QUERY_TOKEN_REC_DESC);
+        this.queryTokenAppender.reset(queryTokenFrame, true);
     }
 
     public void reset() {
@@ -116,7 +114,7 @@ public abstract class AbstractTOccurrenceSearcher implements IInvertedIndexSearc
     }
 
     public IFrameTupleAccessor createResultFrameTupleAccessor() {
-        return new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), searchResult.getTypeTraits());
+        return new FixedSizeFrameTupleAccessor(ctx.getInitialFrameSize(), searchResult.getTypeTraits());
     }
 
     public ITupleReference createResultFrameTupleReference() {
@@ -144,8 +142,10 @@ public abstract class AbstractTOccurrenceSearcher implements IInvertedIndexSearc
             ByteBuffer testBuf = buffer.get(i);
             resultFrameTupleAcc.reset(testBuf);
             for (int j = 0; j < resultFrameTupleAcc.getTupleCount(); j++) {
-                strBuffer.append(IntegerPointable.getInteger(resultFrameTupleAcc.getBuffer().array(), resultFrameTupleAcc.getFieldStartOffset(j, 0)) + ",");
-                strBuffer.append(IntegerPointable.getInteger(resultFrameTupleAcc.getBuffer().array(), resultFrameTupleAcc.getFieldStartOffset(j, 1)) + " ");
+                strBuffer.append(IntegerPointable.getInteger(resultFrameTupleAcc.getBuffer().array(),
+                        resultFrameTupleAcc.getFieldStartOffset(j, 0)) + ",");
+                strBuffer.append(IntegerPointable.getInteger(resultFrameTupleAcc.getBuffer().array(),
+                        resultFrameTupleAcc.getFieldStartOffset(j, 1)) + " ");
             }
         }
         System.out.println(strBuffer.toString());

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
index b7456e6..1e68099 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
@@ -24,7 +24,6 @@ import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
@@ -64,7 +63,7 @@ public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher
             lowerBoundTupleBuilder.addFieldEndOffset();
             lowerBoundTuple.reset(lowerBoundTupleBuilder.getFieldEndOffsets(), lowerBoundTupleBuilder.getByteArray());
             // Only needed for setting the number of fields in searchKey.
-            searchKey.reset(queryTokenAccessor, 0);
+            searchKey.reset(queryTokenAppender, 0);
             fullLowSearchKey.reset();
             fullLowSearchKey.addTuple(searchKey);
             fullLowSearchKey.addTuple(lowerBoundTuple);
@@ -75,7 +74,7 @@ public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher
             upperBoundTupleBuilder.addFieldEndOffset();
             upperBoundTuple.reset(upperBoundTupleBuilder.getFieldEndOffsets(), upperBoundTupleBuilder.getByteArray());
             // Only needed for setting the number of fields in searchKey.
-            searchKey.reset(queryTokenAccessor, 0);
+            searchKey.reset(queryTokenAppender, 0);
             fullHighSearchKey.reset();
             fullHighSearchKey.addTuple(searchKey);
             fullHighSearchKey.addTuple(upperBoundTuple);
@@ -93,7 +92,7 @@ public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher
         }
 
         tokenizeQuery(searchPred);
-        short numQueryTokens = (short) queryTokenAccessor.getTupleCount();
+        short numQueryTokens = (short) queryTokenAppender.getTupleCount();
 
         IInvertedIndexSearchModifier searchModifier = searchPred.getSearchModifier();
         short numTokensLowerBound = searchModifier.getNumTokensLowerBound(numQueryTokens);
@@ -109,7 +108,7 @@ public class PartitionedTOccurrenceSearcher extends AbstractTOccurrenceSearcher
         partitions.reset(numTokensLowerBound, numTokensUpperBound);
         cursorsOrderedByTokens.clear();
         for (int i = 0; i < numQueryTokens; i++) {
-            searchKey.reset(queryTokenAccessor, i);
+            searchKey.reset(queryTokenAppender, i);
             if (!partInvIndex.openInvertedListPartitionCursors(this, ictx, numTokensLowerBound, numTokensUpperBound,
                     partitions, cursorsOrderedByTokens)) {
                 maxCountPossible--;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
index c4056c5..4af9546 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
@@ -54,8 +54,8 @@ public class SearchResult {
         // Integer for counting occurrences.
         typeTraits[invListFields.length] = IntegerPointable.TYPE_TRAITS;
         this.ctx = ctx;
-        appender = new FixedSizeFrameTupleAppender(ctx.getFrameSize(), typeTraits);
-        accessor = new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), typeTraits);
+        appender = new FixedSizeFrameTupleAppender(ctx.getInitialFrameSize(), typeTraits);
+        accessor = new FixedSizeFrameTupleAccessor(ctx.getInitialFrameSize(), typeTraits);
         tuple = new FixedSizeTupleReference(typeTraits);
         buffers.add(ctx.allocateFrame());
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
index 348ef75..c61a86e 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
@@ -38,12 +38,12 @@ public class TOccurrenceSearcher extends AbstractTOccurrenceSearcher {
     public void search(OnDiskInvertedIndexSearchCursor resultCursor, InvertedIndexSearchPredicate searchPred,
             IIndexOperationContext ictx) throws HyracksDataException, IndexException {
         tokenizeQuery(searchPred);
-        int numQueryTokens = queryTokenAccessor.getTupleCount();
+        int numQueryTokens = queryTokenAppender.getTupleCount();
 
         invListCursors.clear();
         invListCursorCache.reset();
         for (int i = 0; i < numQueryTokens; i++) {
-            searchKey.reset(queryTokenAccessor, i);
+            searchKey.reset(queryTokenAppender, i);
             IInvertedListCursor invListCursor = invListCursorCache.getNext();
             invIndex.openInvertedListCursor(invListCursor, searchKey, ictx);
             invListCursors.add(invListCursor);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
index e1a196a..03ff58f 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
@@ -27,10 +27,12 @@ import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
 import edu.uci.ics.hyracks.api.resources.IDeallocatable;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
 
 public class TestJobletContext implements IHyracksJobletContext {
     private final int frameSize;
     private final INCApplicationContext appContext;
+    private final FrameManager frameManger;
     private JobId jobId;
     private WorkspaceFileFactory fileFactory;
 
@@ -39,10 +41,23 @@ public class TestJobletContext implements IHyracksJobletContext {
         this.appContext = appContext;
         this.jobId = jobId;
         fileFactory = new WorkspaceFileFactory(this, (IOManager) getIOManager());
+        this.frameManger = new FrameManager(frameSize);
     }
 
-    public ByteBuffer allocateFrame() {
-        return ByteBuffer.allocate(frameSize);
+    ByteBuffer allocateFrame() throws HyracksDataException {
+        return frameManger.allocateFrame();
+    }
+
+    public ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+        return frameManger.allocateFrame(bytes);
+    }
+
+    ByteBuffer reallocateFrame(ByteBuffer tobeDeallocate, int newFrameSizeInBytes, boolean copyOldData) throws HyracksDataException {
+        return frameManger.reallocateFrame(tobeDeallocate, newFrameSizeInBytes, copyOldData);
+    }
+
+    void deallocateFrames(int bytes) {
+        frameManger.deallocateFrames(bytes);
     }
 
     public int getFrameSize() {
@@ -106,4 +121,5 @@ public class TestJobletContext implements IHyracksJobletContext {
     public ClassLoader getClassLoader() {
         return this.getClass().getClassLoader();
     }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
index 2e4c812..9e395ab 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
@@ -43,16 +43,29 @@ public class TestTaskContext implements IHyracksTaskContext {
     }
 
     @Override
-    public ByteBuffer allocateFrame() {
+    public ByteBuffer allocateFrame() throws HyracksDataException {
         return jobletContext.allocateFrame();
     }
-    
+
+    @Override
+    public ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+        return jobletContext.allocateFrame(bytes);
+    }
+
+    @Override
+    public ByteBuffer reallocateFrame(ByteBuffer tobeDeallocate, int newSizeInBytes, boolean copyOldData)
+            throws HyracksDataException {
+        return jobletContext.reallocateFrame(tobeDeallocate,newSizeInBytes, copyOldData);
+
+    }
+
     @Override
-    public void deallocateFrames(int frameCount) {
+    public void deallocateFrames(int bytes) {
+        jobletContext.deallocateFrames(bytes);
     }
 
     @Override
-    public int getFrameSize() {
+    public int getInitialFrameSize() {
         return jobletContext.getFrameSize();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
index 2c06c20..72e410a 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
@@ -15,13 +15,14 @@
 package edu.uci.ics.hyracks.storage.am.btree;
 
 import java.io.DataOutput;
-import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.logging.Level;
 
 import org.junit.Test;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -110,16 +111,16 @@ public class BTreeStatsTest extends AbstractBTreeTest {
             LOGGER.info("INSERTING INTO TREE");
         }
 
-        ByteBuffer frame = ctx.allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        IFrame frame = new VSizeFrame(ctx);
+        FrameTupleAppender appender = new FrameTupleAppender();
         ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
         DataOutput dos = tb.getDataOutput();
 
         ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
-        accessor.reset(frame);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(recDesc);
+        accessor.reset(frame.getBuffer());
         FrameTupleReference tuple = new FrameTupleReference();
 
         ITreeIndexAccessor indexAccessor = btree.createAccessor(TestOperationCallback.INSTANCE,

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
index a86238a..49b9a8a 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
@@ -16,14 +16,15 @@
 package edu.uci.ics.hyracks.storage.am.btree;
 
 import java.io.DataOutput;
-import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.logging.Level;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -69,8 +70,8 @@ public class FieldPrefixNSMTest extends AbstractBTreeTest {
             }
         }
 
-        ByteBuffer buf = ctx.allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        IFrame buf = new VSizeFrame(ctx);
+        FrameTupleAppender appender = new FrameTupleAppender(buf);
         ArrayTupleBuilder tb = new ArrayTupleBuilder(3);
         DataOutput dos = tb.getDataOutput();
 
@@ -78,8 +79,8 @@ public class FieldPrefixNSMTest extends AbstractBTreeTest {
         ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
                 IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
         RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
-        IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
-        accessor.reset(buf);
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(recDesc);
+        accessor.reset(buf.getBuffer());
         FrameTupleReference tuple = new FrameTupleReference();
 
         tb.reset();


[03/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeapTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeapTest.java
new file mode 100644
index 0000000..bcdabd1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeapTest.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+public class MinHeapTest extends AbstracHeapTest{
+
+    @Test
+    public void testInitialMinHeap() {
+        int capacity = 10;
+        MinHeap minHeap = new MinHeap(new IntFactory(), capacity);
+        assertTrue(minHeap.isEmpty());
+        assertEquals(0, minHeap.getNumEntries());
+    }
+
+    @Test
+    public void testInsertSmallAmountElements() {
+        int capacity = 10;
+        MinHeap minHeap = new MinHeap(new IntFactory(), capacity);
+        for (int i = 0; i < capacity; i++) {
+            minHeap.insert(new Int(capacity - i));
+        }
+        assertEquals(capacity, minHeap.getNumEntries());
+        assertFalse(minHeap.isEmpty());
+
+        assertGetMinHeapIsSorted(minHeap);
+
+        for (int i = 0; i < capacity; i++) {
+            minHeap.insert(new Int(random.nextInt()));
+        }
+        assertEquals(capacity, minHeap.getNumEntries());
+        assertFalse(minHeap.isEmpty());
+        assertGetMinHeapIsSorted(minHeap);
+    }
+
+    @Test
+    public void testInsertLargerThanCapacityElements() {
+        int capacity = 10;
+        MinHeap minHeap = new MinHeap(new IntFactory(), capacity);
+        for (int i = 0; i < capacity; i++) {
+            minHeap.insert(new Int(capacity - i));
+        }
+        assertEquals(capacity, minHeap.getNumEntries());
+        assertFalse(minHeap.isEmpty());
+        assertGetMinHeapIsSorted(minHeap);
+
+        for (int i = 0; i < capacity * 10; i++) {
+            minHeap.insert(new Int(random.nextInt()));
+        }
+        assertEquals(capacity * 10, minHeap.getNumEntries());
+        assertFalse(minHeap.isEmpty());
+        assertGetMinHeapIsSorted(minHeap);
+
+    }
+
+    @Test
+    public void testReplaceMin() {
+        int capacity = 10;
+        MinHeap minHeap = new MinHeap(new IntFactory(), capacity);
+        for (int i = 0; i < capacity; i++) {
+            minHeap.insert(new Int(i));
+        }
+        assertEquals(capacity, minHeap.getNumEntries());
+        assertFalse(minHeap.isEmpty());
+
+        for (int i = capacity; i < capacity * 2; i++) {
+            minHeap.replaceMin(new Int(i));
+        }
+        assertEquals(capacity, minHeap.getNumEntries());
+        assertFalse(minHeap.isEmpty());
+
+        Int minI = new Int();
+        Int peekI = new Int();
+        int i = 0;
+        while (!minHeap.isEmpty()) {
+            minHeap.peekMin(peekI);
+            minHeap.getMin(minI);
+            assertTrue(peekI.compareTo(minI) == 0);
+            assertEquals(i++ + capacity, minI.i);
+        }
+    }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeapTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeapTest.java
new file mode 100644
index 0000000..1f16e08
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeapTest.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+public class MinMaxHeapTest extends AbstracHeapTest {
+
+    @Test
+    public void testInitialMinMaxHeap() {
+        int capacity = 10;
+        MinMaxHeap minHeap = new MinMaxHeap(new IntFactory(), capacity);
+        assertTrue(minHeap.isEmpty());
+        assertEquals(0, minHeap.getNumEntries());
+    }
+
+    @Test
+    public void testInsertElements() {
+        int capacity = 10;
+        MinMaxHeap minMaxHeap = new MinMaxHeap(new IntFactory(), capacity);
+        for (int i = 0; i < capacity * 10; i++) {
+            minMaxHeap.insert(new Int(random.nextInt()));
+        }
+        assertEquals(capacity * 10, minMaxHeap.getNumEntries());
+        assertFalse(minMaxHeap.isEmpty());
+        assertGetMinHeapIsSorted(minMaxHeap);
+
+        for (int i = 0; i < capacity * 10; i++) {
+            minMaxHeap.insert(new Int(random.nextInt()));
+        }
+        assertEquals(capacity * 10, minMaxHeap.getNumEntries());
+        assertGetMaxHeapIsSorted(minMaxHeap);
+    }
+
+    @Test
+    public void testReplaceMin() {
+        int capacity = 10;
+        MinMaxHeap minMaxHeap = new MinMaxHeap(new IntFactory(), capacity);
+        for (int i = 0; i < capacity; i++) {
+            minMaxHeap.insert(new Int(i));
+        }
+        assertEquals(capacity, minMaxHeap.getNumEntries());
+        assertFalse(minMaxHeap.isEmpty());
+
+        for (int i = capacity; i < capacity * 2; i++) {
+            minMaxHeap.replaceMin(new Int(i));
+        }
+        assertEquals(capacity, minMaxHeap.getNumEntries());
+        assertFalse(minMaxHeap.isEmpty());
+
+        Int minI = new Int();
+        Int peekI = new Int();
+        int i = 0;
+        while (!minMaxHeap.isEmpty()) {
+            minMaxHeap.peekMin(peekI);
+            minMaxHeap.getMin(minI);
+            assertTrue(peekI.compareTo(minI) == 0);
+            assertEquals(i++ + capacity, minI.i);
+        }
+    }
+
+    @Test
+    public void testReplaceMax() {
+        int capacity = 10;
+        MinMaxHeap minMaxHeap = new MinMaxHeap(new IntFactory(), capacity);
+        for (int i = 0; i < capacity; i++) {
+            minMaxHeap.insert(new Int(i + capacity));
+        }
+        assertEquals(capacity, minMaxHeap.getNumEntries());
+        assertFalse(minMaxHeap.isEmpty());
+
+        Int maxI = new Int();
+        for (int i = capacity; i < capacity * 2; i++) {
+            minMaxHeap.peekMax(maxI);
+            minMaxHeap.replaceMax(new Int(i - capacity));
+        }
+        assertEquals(capacity, minMaxHeap.getNumEntries());
+        assertFalse(minMaxHeap.isEmpty());
+
+        System.out.println();
+        Int peekI = new Int();
+        int i = 0;
+        while (!minMaxHeap.isEmpty()) {
+            minMaxHeap.peekMax(peekI);
+            minMaxHeap.getMax(maxI);
+            assertTrue(peekI.compareTo(maxI) == 0);
+            assertEquals(capacity - i - 1, maxI.i);
+            i++;
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/util/MathTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/util/MathTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/util/MathTest.java
new file mode 100644
index 0000000..332ac98
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/util/MathTest.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.util;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.Random;
+
+import org.junit.Test;
+
+public class MathTest {
+
+    @Test
+    public void testLog2() {
+        Random random = new Random(System.currentTimeMillis());
+        for (int i = 0; i < 31; i++) {
+            assertTrue(MathUtil.log2Floor((int) Math.pow(2, i)) == i);
+            for(int x = 0; x < 10; x++){
+                float extra = random.nextFloat();
+                while (extra >= 1.0){
+                    extra = random.nextFloat();
+                }
+                assertTrue(MathUtil.log2Floor((int) Math.pow(2, i + extra)) == i);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
index ed4cf0c..c869362 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
@@ -16,10 +16,10 @@
 package edu.uci.ics.hyracks.examples.btree.helper;
 
 import java.io.DataOutput;
-import java.nio.ByteBuffer;
 import java.util.HashSet;
 import java.util.Random;
 
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -28,7 +28,6 @@ import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 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.comm.util.FrameUtils;
 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.base.AbstractSingleActivityOperatorDescriptor;
@@ -61,8 +60,7 @@ public class DataGenOperatorDescriptor extends AbstractSingleActivityOperatorDes
     public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
             IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
 
-        final ByteBuffer outputFrame = ctx.allocateFrame();
-        final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        final FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
         final RecordDescriptor recDesc = recordDescriptors[0];
         final ArrayTupleBuilder tb = new ArrayTupleBuilder(recDesc.getFields().length);
         final Random rnd = new Random(randomSeed);
@@ -79,7 +77,6 @@ public class DataGenOperatorDescriptor extends AbstractSingleActivityOperatorDes
             public void initialize() throws HyracksDataException {
                 writer.open();
                 try {
-                    appender.reset(outputFrame, true);
                     for (int i = 0; i < numRecords; i++) {
                         tb.reset();
                         for (int j = 0; j < recDesc.getFieldCount(); j++) {
@@ -87,14 +84,13 @@ public class DataGenOperatorDescriptor extends AbstractSingleActivityOperatorDes
                         }
 
                         if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                            FrameUtils.flushFrame(outputFrame, writer);
-                            appender.reset(outputFrame, true);
+                            appender.flush(writer, true);
                             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                                 throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
                             }
                         }
                     }
-                    FrameUtils.flushFrame(outputFrame, writer);
+                    appender.flush(writer, true);
                 } catch (Exception e) {
                     writer.fail();
                     throw new HyracksDataException(e);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/.gitignore
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/.gitignore b/hyracks/hyracks-examples/hyracks-integration-tests/.gitignore
new file mode 100644
index 0000000..be303ea
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/.gitignore
@@ -0,0 +1,3 @@
+primary*/
+secondary*/
+inv*/

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
index 29be04b..5268150 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
@@ -24,8 +24,10 @@ import java.util.logging.Logger;
 
 import org.junit.Test;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
@@ -48,12 +50,12 @@ public class SerializationDeserializationTest {
         private final IHyracksTaskContext ctx;
         private static final int FRAME_SIZE = 32768;
         private RecordDescriptor rDes;
-        private List<ByteBuffer> buffers;
+        private List<IFrame> buffers;
 
         public SerDeserRunner(RecordDescriptor rDes) throws HyracksException {
             ctx = TestUtils.create(FRAME_SIZE);
             this.rDes = rDes;
-            buffers = new ArrayList<ByteBuffer>();
+            buffers = new ArrayList<>();
         }
 
         public IOpenableDataWriter<Object[]> createWriter() throws HyracksDataException {
@@ -64,8 +66,8 @@ public class SerializationDeserializationTest {
 
                 @Override
                 public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    ByteBuffer toBuf = ctx.allocateFrame();
-                    toBuf.put(buffer);
+                    IFrame toBuf = new VSizeFrame(ctx);
+                    toBuf.getBuffer().put(buffer);
                     buffers.add(toBuf);
                 }
 
@@ -89,12 +91,12 @@ public class SerializationDeserializationTest {
                 }
 
                 @Override
-                public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                public boolean nextFrame(IFrame frame) throws HyracksDataException {
                     if (i < buffers.size()) {
-                        ByteBuffer buf = buffers.get(i);
-                        buf.flip();
-                        buffer.put(buf);
-                        buffer.flip();
+                        IFrame buf = buffers.get(i);
+                        buf.getBuffer().flip();
+                        frame.getBuffer().put(buf.getBuffer());
+                        frame.getBuffer().flip();
                         ++i;
                         return true;
                     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 1150cf3..30f89e2 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -15,10 +15,11 @@
 package edu.uci.ics.hyracks.tests.integration;
 
 import java.io.BufferedReader;
+import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileReader;
+import java.io.FileWriter;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
@@ -34,6 +35,7 @@ import org.junit.rules.TemporaryFolder;
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
 import edu.uci.ics.hyracks.api.dataset.ResultSetId;
@@ -45,6 +47,7 @@ import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
@@ -138,12 +141,11 @@ public abstract class AbstractIntegrationTest {
         hcc.waitForCompletion(jobId);
     }
 
+
     protected List<String> readResults(JobSpecification spec, JobId jobId, ResultSetId resultSetId) throws Exception {
         int nReaders = 1;
-        ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
-        resultBuffer.clear();
 
-        IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+        IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor();
 
         IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
         IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, resultSetId);
@@ -151,16 +153,18 @@ public abstract class AbstractIntegrationTest {
         List<String> resultRecords = new ArrayList<String>();
         ByteBufferInputStream bbis = new ByteBufferInputStream();
 
-        int readSize = reader.read(resultBuffer);
+        FrameManager resultDisplayFrameMgr = new FrameManager(spec.getFrameSize());
+        VSizeFrame frame = new VSizeFrame(resultDisplayFrameMgr);
+        int readSize = reader.read(frame);
 
         while (readSize > 0) {
 
             try {
-                frameTupleAccessor.reset(resultBuffer);
+                frameTupleAccessor.reset(frame.getBuffer());
                 for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
                     int start = frameTupleAccessor.getTupleStartOffset(tIndex);
                     int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
-                    bbis.setByteBuffer(resultBuffer, start);
+                    bbis.setByteBuffer(frame.getBuffer(), start);
                     byte[] recordBytes = new byte[length];
                     bbis.read(recordBytes, 0, length);
                     resultRecords.add(new String(recordBytes, 0, length));
@@ -169,8 +173,7 @@ public abstract class AbstractIntegrationTest {
                 bbis.close();
             }
 
-            resultBuffer.clear();
-            readSize = reader.read(resultBuffer);
+            readSize = reader.read(frame);
         }
         return resultRecords;
     }
@@ -198,6 +201,22 @@ public abstract class AbstractIntegrationTest {
         return true;
     }
 
+    protected void runTestAndStoreResult(JobSpecification spec, File file) throws Exception {
+        JobId jobId = executeTest(spec);
+
+        BufferedWriter output = new BufferedWriter(new FileWriter(file));
+        List<String> results;
+        for (int i = 0; i < spec.getResultSetIds().size(); i++) {
+            results = readResults(spec, jobId, spec.getResultSetIds().get(i));
+            for(String str : results) {
+                output.write(str);
+            }
+        }
+        output.close();
+
+        hcc.waitForCompletion(jobId);
+    }
+
     protected File createTempFile() throws IOException {
         File tempFile = File.createTempFile(getClass().getName(), ".tmp", outputFolder.getRoot());
         if (LOGGER.isLoggable(Level.INFO)) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
index 970f2fe..602e193 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
@@ -16,7 +16,6 @@ package edu.uci.ics.hyracks.tests.integration;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
@@ -33,6 +32,7 @@ import org.junit.rules.TemporaryFolder;
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -44,6 +44,7 @@ import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
 import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
 import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
 import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
@@ -66,7 +67,8 @@ public abstract class AbstractMultiNCIntegrationTest {
     public TemporaryFolder outputFolder = new TemporaryFolder();
 
     public AbstractMultiNCIntegrationTest() {
-        outputFiles = new ArrayList<File>();;
+        outputFiles = new ArrayList<File>();
+        ;
     }
 
     @BeforeClass
@@ -124,10 +126,10 @@ public abstract class AbstractMultiNCIntegrationTest {
 
         int nReaders = 1;
 
-        ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
-        resultBuffer.clear();
+        FrameManager resultDisplayFrameMgr = new FrameManager(spec.getFrameSize());
+        VSizeFrame resultFrame = new VSizeFrame(resultDisplayFrameMgr);
 
-        IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+        IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor();
 
         IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
         IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, spec.getResultSetIds().get(0));
@@ -135,16 +137,16 @@ public abstract class AbstractMultiNCIntegrationTest {
         JSONArray resultRecords = new JSONArray();
         ByteBufferInputStream bbis = new ByteBufferInputStream();
 
-        int readSize = reader.read(resultBuffer);
+        int readSize = reader.read(resultFrame);
 
         while (readSize > 0) {
 
             try {
-                frameTupleAccessor.reset(resultBuffer);
+                frameTupleAccessor.reset(resultFrame.getBuffer());
                 for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
                     int start = frameTupleAccessor.getTupleStartOffset(tIndex);
                     int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
-                    bbis.setByteBuffer(resultBuffer, start);
+                    bbis.setByteBuffer(resultFrame.getBuffer(), start);
                     byte[] recordBytes = new byte[length];
                     bbis.read(recordBytes, 0, length);
                     resultRecords.put(new String(recordBytes, 0, length));
@@ -157,8 +159,7 @@ public abstract class AbstractMultiNCIntegrationTest {
                 }
             }
 
-            resultBuffer.clear();
-            readSize = reader.read(resultBuffer);
+            readSize = reader.read(resultFrame);
         }
 
         hcc.waitForCompletion(jobId);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
index a2ef99a..9b77ec5 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
@@ -42,9 +42,10 @@ 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.PlainFileWriterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.LimitOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.sort.OptimizedExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.TopKSorterOperatorDescriptor;
 import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
 
 public class OptimizedSortMergeTest extends AbstractIntegrationTest {
@@ -72,17 +73,22 @@ public class OptimizedSortMergeTest extends AbstractIntegrationTest {
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
-        OptimizedExternalSortOperatorDescriptor sorter = new OptimizedExternalSortOperatorDescriptor(spec, 4,
-                new int[] { 1, 0 }, new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
+        int outputLimit = 5; // larger than the total record numbers.
+        TopKSorterOperatorDescriptor sorter = new TopKSorterOperatorDescriptor(spec, 4,
+                outputLimit, new int[] { 1, 0 }, null, new IBinaryComparatorFactory[] {
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
         ResultSetId rsId = new ResultSetId(1);
         spec.addResultSetId(rsId);
 
-        IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false, false,
-                ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+        File file = File.createTempFile(getClass().getName(), ".tmp");
+        IFileSplitProvider outputSplitProvider = new ConstantFileSplitProvider(
+                new FileSplit[] { new FileSplit(NC1_ID, file.getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outputSplitProvider, "|");
+
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
 
         spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -98,6 +104,7 @@ public class OptimizedSortMergeTest extends AbstractIntegrationTest {
                         new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, printer, 0);
 
         runTest(spec);
+        System.out.println("Result write into :" + file.getAbsolutePath());
     }
 
     @Test
@@ -123,11 +130,11 @@ public class OptimizedSortMergeTest extends AbstractIntegrationTest {
                         UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
 
-        int outputLimit = 200;
-        OptimizedExternalSortOperatorDescriptor sorter = new OptimizedExternalSortOperatorDescriptor(spec, 4,
-                outputLimit, new int[] { 1, 0 }, new IBinaryComparatorFactory[] {
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
-                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
+        int outputLimit = 20;
+        TopKSorterOperatorDescriptor sorter = new TopKSorterOperatorDescriptor(spec, 4,
+                outputLimit, new int[] { 1, 0 }, null, new IBinaryComparatorFactory[] {
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
         PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
 
         LimitOperatorDescriptor filter = new LimitOperatorDescriptor(spec, ordersDesc, outputLimit);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/VSizeFrameSortMergeTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/VSizeFrameSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/VSizeFrameSortMergeTest.java
new file mode 100644
index 0000000..212a99c
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/VSizeFrameSortMergeTest.java
@@ -0,0 +1,118 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.integration;
+
+import java.io.File;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+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.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.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.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+
+public class VSizeFrameSortMergeTest extends AbstractIntegrationTest {
+
+    public static String[] INPUTS = { "data/tpch0.001/orders-part1.tbl", "data/tpch0.001/orders-part2.tbl" };
+
+    FileSplit[] ordersSplits = new FileSplit[] {
+            new FileSplit(NC1_ID, new FileReference(new File(INPUTS[0]))),
+            new FileSplit(NC2_ID, new FileReference(new File(INPUTS[1]))) };
+    IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+    RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE });
+
+    @Test
+    public void sortNormalMergeTest() throws Exception {
+        sortTask(1024, 4);
+        sortTask(256, 4);
+    }
+
+    @Test
+    public void sortLargeMergeTest() throws Exception {
+        sortTask(32, 128);
+        sortTask(16, 256);
+        sortTask(16, 10240);
+    }
+
+    public void sortTask(int frameSize, int frameLimit) throws Exception {
+        JobSpecification spec = new JobSpecification();
+
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
+        //                PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID );
+
+        spec.setFrameSize(frameSize);
+        ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, frameLimit, new int[] { 1, 0 },
+                new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
+
+        File file = File.createTempFile(getClass().getName(), ".tmp");
+
+        IFileSplitProvider outputSplitProvider = new ConstantFileSplitProvider(
+                new FileSplit[] { new FileSplit(NC1_ID, file.getAbsolutePath()) });
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outputSplitProvider, "|");
+
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
+
+        spec.connect(
+                new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(new int[] {
+                        1, 0 }, new IBinaryHashFunctionFactory[] {
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+                        PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
+                        new IBinaryComparatorFactory[] {
+                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+                                PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                        new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, printer, 0);
+
+        spec.addRoot(printer);
+        runTest(spec);
+        System.out.println("Result write into :" + file.getAbsolutePath());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/AbstractRunGeneratorTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/AbstractRunGeneratorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/AbstractRunGeneratorTest.java
new file mode 100644
index 0000000..c4faa1a
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/AbstractRunGeneratorTest.java
@@ -0,0 +1,279 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.DataInputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+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.ByteBufferInputStream;
+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.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.GroupFrameAccessor;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.GroupVSizeFrame;
+import edu.uci.ics.hyracks.test.support.TestUtils;
+
+public abstract class AbstractRunGeneratorTest {
+    static TestUtils testUtils = new TestUtils();
+    static ISerializerDeserializer[] SerDers = new ISerializerDeserializer[] {
+            IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+    static RecordDescriptor RecordDesc = new RecordDescriptor(SerDers);
+    static Random GRandom = new Random(System.currentTimeMillis());
+    static int[] SortFields = new int[] { 0, 1 };
+    static IBinaryComparatorFactory[] ComparatorFactories = new IBinaryComparatorFactory[] {
+            PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+            PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
+
+    static void assertMaxFrameSizesAreAllEqualsTo(List<RunAndMaxFrameSizePair> maxSize, int pageSize) {
+        for (int i = 0; i < maxSize.size(); i++) {
+            assertTrue(maxSize.get(i).maxFrameSize == pageSize);
+        }
+    }
+
+    abstract AbstractSortRunGenerator getSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int numOfInputRecord)
+            throws HyracksDataException;
+
+    protected List<RunAndMaxFrameSizePair> testSortRecords(int pageSize, int frameLimit, int numRuns, int minRecordSize,
+            int maxRecordSize, HashMap<Integer, String> specialData) throws HyracksDataException {
+        IHyracksTaskContext ctx = testUtils.create(pageSize);
+
+        HashMap<Integer, String> keyValuePair = new HashMap<>();
+        List<IFrame> frameList = new ArrayList<>();
+        prepareData(ctx, frameList, pageSize * frameLimit * numRuns, minRecordSize, maxRecordSize,
+                specialData, keyValuePair);
+        AbstractSortRunGenerator runGenerator = getSortRunGenerator(ctx, frameLimit, keyValuePair.size());
+        runGenerator.open();
+        for (IFrame frame : frameList) {
+            runGenerator.nextFrame(frame.getBuffer());
+        }
+        runGenerator.close();
+        matchResult(ctx, runGenerator.getRuns(), keyValuePair);
+        return runGenerator.getRuns();
+    }
+
+    static void matchResult(IHyracksTaskContext ctx, List<RunAndMaxFrameSizePair> runs,
+            Map<Integer, String> keyValuePair) throws HyracksDataException {
+        IFrame frame = new VSizeFrame(ctx);
+        FrameTupleAccessor fta = new FrameTupleAccessor(RecordDesc);
+
+        HashMap<Integer, String> copyMap = new HashMap<>(keyValuePair);
+        assertReadSorted(runs, fta, frame, copyMap);
+
+        HashMap<Integer, String> copyMap2 = new HashMap<>(keyValuePair);
+        int maxFrameSizes = 0;
+        for (RunAndMaxFrameSizePair run : runs) {
+            maxFrameSizes = Math.max(maxFrameSizes, run.maxFrameSize);
+        }
+        GroupVSizeFrame gframe = new GroupVSizeFrame(ctx, maxFrameSizes);
+        GroupFrameAccessor gfta = new GroupFrameAccessor(ctx.getInitialFrameSize(), RecordDesc);
+        assertReadSorted(runs, gfta, gframe, copyMap2);
+    }
+
+    static int assertFTADataIsSorted(IFrameTupleAccessor fta, Map<Integer, String> keyValuePair, int preKey)
+            throws HyracksDataException {
+
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+        DataInputStream di = new DataInputStream(bbis);
+        for (int i = 0; i < fta.getTupleCount(); i++) {
+            bbis.setByteBuffer(fta.getBuffer(),
+                    fta.getTupleStartOffset(i) + fta.getFieldStartOffset(i, 0) + fta.getFieldSlotsLength());
+            int key = (int) RecordDesc.getFields()[0].deserialize(di);
+            bbis.setByteBuffer(fta.getBuffer(),
+                    fta.getTupleStartOffset(i) + fta.getFieldStartOffset(i, 1) + fta.getFieldSlotsLength());
+            String value = (String) RecordDesc.getFields()[1].deserialize(di);
+
+            if (!keyValuePair.get(key).equals(value)) {
+                assertTrue(false);
+            }
+            keyValuePair.remove(key);
+            assertTrue(key >= preKey);
+            preKey = key;
+        }
+        return preKey;
+    }
+
+    static void assertReadSorted(List<RunAndMaxFrameSizePair> runs, IFrameTupleAccessor fta, IFrame frame,
+            Map<Integer, String> keyValuePair) throws HyracksDataException {
+
+        assertTrue(runs.size() > 0);
+        for (RunAndMaxFrameSizePair run : runs) {
+            run.run.open();
+            int preKey = Integer.MIN_VALUE;
+            while (run.run.nextFrame(frame)) {
+                fta.reset(frame.getBuffer());
+                preKey = assertFTADataIsSorted(fta, keyValuePair, preKey);
+            }
+            run.run.close();
+        }
+        assertTrue(keyValuePair.isEmpty());
+    }
+
+    static void prepareData(IHyracksTaskContext ctx, List<IFrame> frameList, int minDataSize, int minRecordSize,
+            int maxRecordSize, Map<Integer, String> specialData, Map<Integer, String> keyValuePair)
+            throws HyracksDataException {
+
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(RecordDesc.getFieldCount());
+        FrameTupleAppender appender = new FrameTupleAppender();
+
+        int datasize = 0;
+        if (specialData != null) {
+            for (Map.Entry<Integer, String> entry : specialData.entrySet()) {
+                tb.reset();
+                tb.addField(IntegerSerializerDeserializer.INSTANCE, entry.getKey());
+                tb.addField(UTF8StringSerializerDeserializer.INSTANCE, entry.getValue());
+
+                VSizeFrame frame = new VSizeFrame(ctx, FrameHelper
+                        .calcAlignedFrameSizeToStore(tb.getFieldEndOffsets().length, tb.getSize(), ctx.getInitialFrameSize()));
+                appender.reset(frame, true);
+                assertTrue(appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize()));
+                frameList.add(frame);
+                datasize += frame.getFrameSize();
+            }
+            keyValuePair.putAll(specialData);
+        }
+
+        VSizeFrame frame = new VSizeFrame(ctx, ctx.getInitialFrameSize());
+        appender.reset(frame, true);
+        while (datasize < minDataSize) {
+            tb.reset();
+            int key = GRandom.nextInt(minDataSize + 1);
+            if (!keyValuePair.containsKey(key)) {
+                String value = generateRandomRecord(minRecordSize, maxRecordSize);
+                tb.addField(IntegerSerializerDeserializer.INSTANCE, key);
+                tb.addField(UTF8StringSerializerDeserializer.INSTANCE, value);
+
+                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                    frameList.add(frame);
+                    datasize += frame.getFrameSize();
+                    frame = new VSizeFrame(ctx, FrameHelper
+                            .calcAlignedFrameSizeToStore(tb.getFieldEndOffsets().length, tb.getSize(),
+                                    ctx.getInitialFrameSize()));
+                    appender.reset(frame, true);
+                    assertTrue(appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize()));
+                }
+
+                keyValuePair.put(key, value);
+            }
+        }
+        if (appender.getTupleCount() > 0) {
+            frameList.add(frame);
+        }
+
+    }
+
+    static String generateRandomRecord(int minRecordSize, int maxRecordSize)
+            throws HyracksDataException {
+        int size = GRandom.nextInt(maxRecordSize - minRecordSize + 1) + minRecordSize;
+        return generateRandomFixSizedString(size);
+
+    }
+
+    static String generateRandomFixSizedString(int size) {
+        StringBuilder sb = new StringBuilder(size);
+        for (; size >= 0; --size) {
+            char ch = (char) (GRandom.nextInt(26) + 97);
+            sb.append(ch);
+        }
+        return sb.toString();
+    }
+
+    static HashMap<Integer, String> generateBigObject(int pageSize, int times) {
+        HashMap<Integer, String> map = new HashMap<>(1);
+        for (int i = 1; i < times; i++) {
+            map.put(GRandom.nextInt(), generateRandomFixSizedString(pageSize * i));
+        }
+        return map;
+    }
+
+    @Test
+    public void testAllSmallRecords() throws HyracksDataException {
+        int pageSize = 512;
+        int frameLimit = 4;
+        int numRuns = 2;
+        int minRecordSize = pageSize / 8;
+        int maxRecordSize = pageSize / 8;
+        List<RunAndMaxFrameSizePair> maxSize = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize,
+                maxRecordSize, null);
+        assertMaxFrameSizesAreAllEqualsTo(maxSize, pageSize);
+    }
+
+    @Test
+    public void testAllLargeRecords() throws HyracksDataException {
+        int pageSize = 2048;
+        int frameLimit = 4;
+        int numRuns = 2;
+        int minRecordSize = pageSize;
+        int maxRecordSize = (int) (pageSize * 1.8);
+        List<RunAndMaxFrameSizePair> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize, maxRecordSize,
+                null);
+        assertMaxFrameSizesAreAllEqualsTo(size, pageSize * 2);
+    }
+
+    @Test
+    public void testMixedLargeRecords() throws HyracksDataException {
+        int pageSize = 128;
+        int frameLimit = 4;
+        int numRuns = 4;
+        int minRecordSize = 20;
+        int maxRecordSize = pageSize / 2;
+        HashMap<Integer, String> specialPair = generateBigObject(pageSize, frameLimit - 1);
+        List<RunAndMaxFrameSizePair> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize, maxRecordSize,
+                specialPair);
+
+        int max = 0;
+        for (RunAndMaxFrameSizePair run : size) {
+            max = Math.max(max, run.maxFrameSize);
+        }
+        assertTrue(max == pageSize * (frameLimit - 1));
+    }
+
+    @Test(expected = HyracksDataException.class)
+    public void testTooBigRecordWillThrowException() throws HyracksDataException {
+        int pageSize = 1024;
+        int frameLimit = 8;
+        int numRuns = 8;
+        HashMap<Integer, String> specialPair = generateBigObject(pageSize, frameLimit);
+        int minRecordSize = 10;
+        int maxRecordSize = pageSize / 2;
+        List<RunAndMaxFrameSizePair> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize, maxRecordSize,
+                specialPair);
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/ExternalSortRunGeneratorTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/ExternalSortRunGeneratorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/ExternalSortRunGeneratorTest.java
new file mode 100644
index 0000000..4d7558b
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/ExternalSortRunGeneratorTest.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
+
+public class ExternalSortRunGeneratorTest extends AbstractRunGeneratorTest {
+
+    @Override
+    AbstractSortRunGenerator getSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int numOfInputRecord)
+            throws HyracksDataException {
+        return new ExternalSortRunGenerator(ctx, SortFields, null, ComparatorFactories, RecordDesc,
+                Algorithm.MERGE_SORT, frameLimit);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HeapSortRunGeneratorTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HeapSortRunGeneratorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HeapSortRunGeneratorTest.java
new file mode 100644
index 0000000..00eca70
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HeapSortRunGeneratorTest.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.HeapSortRunGenerator;
+
+public class HeapSortRunGeneratorTest extends AbstractRunGeneratorTest {
+    @Override
+    AbstractSortRunGenerator getSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int numOfInputRecord)
+            throws HyracksDataException {
+        return new HeapSortRunGenerator(ctx, frameLimit, numOfInputRecord, SortFields, null, ComparatorFactories,
+                RecordDesc);
+    }
+
+    @Test
+    public void testTopK(){
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HybridSortRunGenerator.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HybridSortRunGenerator.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HybridSortRunGenerator.java
new file mode 100644
index 0000000..f7ecd5e
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HybridSortRunGenerator.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.HybridTopKSortRunGenerator;
+
+public class HybridSortRunGenerator extends AbstractRunGeneratorTest {
+    @Override
+    AbstractSortRunGenerator getSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int numOfInputRecord)
+            throws HyracksDataException {
+        return new HybridTopKSortRunGenerator(ctx, frameLimit, numOfInputRecord, SortFields, null, ComparatorFactories,
+                RecordDesc);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/RunMergingFrameReaderTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/RunMergingFrameReaderTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/RunMergingFrameReaderTest.java
new file mode 100644
index 0000000..d5355b8
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/RunMergingFrameReaderTest.java
@@ -0,0 +1,409 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.ComparatorFactories;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.GRandom;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.RecordDesc;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.SortFields;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.generateRandomRecord;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.matchResult;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.prepareData;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.testUtils;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+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.ByteBufferInputStream;
+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.sort.Algorithm;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.GroupVSizeFrame;
+
+public class RunMergingFrameReaderTest {
+    static IBinaryComparator[] Comparators = new IBinaryComparator[] {
+            ComparatorFactories[0].createBinaryComparator(),
+            ComparatorFactories[1].createBinaryComparator(),
+    };
+
+    static class TestFrameReader implements IFrameReader {
+
+        private final int pageSize;
+        private final int numFrames;
+        private final int minRecordSize;
+        private final int maxRecordSize;
+        private TreeMap<Integer, String> result = new TreeMap<>();
+        int maxFrameSize;
+
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(RecordDesc.getFieldCount());
+        FrameTupleAppender appender = new FrameTupleAppender();
+        private Iterator<Map.Entry<Integer, String>> iterator;
+        private Map.Entry<Integer, String> lastEntry;
+
+        TestFrameReader(int pageSize, int numFrames, int minRecordSize, int maxRecordSize) {
+            this.pageSize = pageSize;
+            this.numFrames = numFrames;
+            this.minRecordSize = minRecordSize;
+            this.maxRecordSize = maxRecordSize;
+            this.maxFrameSize = pageSize;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            result.clear();
+            int maxTupleSize = prepareSortedData(numFrames * pageSize, minRecordSize, maxRecordSize, null, result);
+            maxFrameSize = FrameHelper.calcAlignedFrameSizeToStore(0, maxTupleSize, pageSize);
+            iterator = result.entrySet().iterator();
+        }
+
+        @Override
+        public boolean nextFrame(IFrame frame) throws HyracksDataException {
+            if (lastEntry == null && !iterator.hasNext()) {
+                return false;
+            }
+            if (lastEntry == null) {
+                lastEntry = iterator.next();
+            }
+            appender.reset(frame, true);
+            while (true) {
+                tb.reset();
+                tb.addField(IntegerSerializerDeserializer.INSTANCE, lastEntry.getKey());
+                tb.addField(UTF8StringSerializerDeserializer.INSTANCE, lastEntry.getValue());
+                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+                    break;
+                } else {
+                    if (iterator.hasNext()) {
+                        lastEntry = iterator.next();
+                    } else {
+                        lastEntry = null;
+                        break;
+                    }
+
+                }
+            }
+            //            printFrame(frame.getBuffer());
+            return true;
+        }
+
+        private void printFrame(ByteBuffer buffer) {
+            FrameTupleAccessor fta = new FrameTupleAccessor(RecordDesc);
+            fta.reset(buffer);
+            fta.prettyPrint();
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+        }
+    }
+
+    static int prepareSortedData(int minDataSize, int minRecordSize, int maxRecordSize,
+            Map<Integer, String> specialData, Map<Integer, String> result) throws HyracksDataException {
+
+        ArrayTupleBuilder tb = new ArrayTupleBuilder(RecordDesc.getFieldCount());
+
+        int datasize = 0;
+        int maxtuple = 0;
+        if (specialData != null) {
+            for (Map.Entry<Integer, String> entry : specialData.entrySet()) {
+                tb.reset();
+                tb.addField(IntegerSerializerDeserializer.INSTANCE, entry.getKey());
+                tb.addField(UTF8StringSerializerDeserializer.INSTANCE, entry.getValue());
+                int size = tb.getSize() + tb.getFieldEndOffsets().length * 4;
+                datasize += size;
+                if (size > maxtuple) {
+                    maxtuple = size;
+                }
+            }
+            result.putAll(specialData);
+        }
+
+        while (datasize < minDataSize) {
+            String value = generateRandomRecord(minRecordSize, maxRecordSize);
+            tb.reset();
+            int key = GRandom.nextInt(datasize + 1);
+            if (!result.containsKey(key)) {
+                tb.addField(IntegerSerializerDeserializer.INSTANCE, key);
+                tb.addField(UTF8StringSerializerDeserializer.INSTANCE, value);
+                int size = tb.getSize() + tb.getFieldEndOffsets().length * 4;
+                datasize += size;
+                if (size > maxtuple) {
+                    maxtuple = size;
+                }
+                if (datasize < minDataSize) {
+                    result.put(key, value);
+                }
+            }
+        }
+
+        return maxtuple;
+    }
+
+    @Test
+    public void testOnlyOneRunShouldMerge() throws HyracksDataException {
+        int pageSize = 128;
+        int numRuns = 1;
+        int numFramesPerRun = 1;
+        int minRecordSize = pageSize / 10;
+        int maxRecordSize = pageSize / 8;
+
+        IHyracksTaskContext ctx = testUtils.create(pageSize);
+        List<Map<Integer, String>> keyValueMapList = new ArrayList<>(numRuns);
+        List<TestFrameReader> readerList = new ArrayList<>(numRuns);
+        List<IFrame> frameList = new ArrayList<>(numRuns);
+        prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+                minRecordSize, maxRecordSize, readerList, frameList, keyValueMapList);
+
+        RunMergingFrameReader reader = new RunMergingFrameReader(ctx, readerList, frameList, SortFields, Comparators,
+                null, RecordDesc);
+        testMergeSucceed(ctx, reader, keyValueMapList);
+    }
+
+    @Test
+    public void testNormalRunMerge() throws HyracksDataException {
+
+        int pageSize = 128;
+        int numRuns = 2;
+        int numFramesPerRun = 2;
+        int minRecordSize = pageSize / 10;
+        int maxRecordSize = pageSize / 8;
+
+        IHyracksTaskContext ctx = testUtils.create(pageSize);
+        List<Map<Integer, String>> keyValueMapList = new ArrayList<>(numRuns);
+        List<TestFrameReader> readerList = new ArrayList<>(numRuns);
+        List<IFrame> frameList = new ArrayList<>(numRuns);
+        prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+                minRecordSize, maxRecordSize, readerList, frameList, keyValueMapList);
+
+        RunMergingFrameReader reader = new RunMergingFrameReader(ctx, readerList, frameList, SortFields, Comparators,
+                null, RecordDesc);
+        testMergeSucceed(ctx, reader, keyValueMapList);
+    }
+
+    @Test
+    public void testNormalRunMergeWithTopK() throws HyracksDataException {
+
+        int pageSize = 128;
+        int numRuns = 2;
+        int numFramesPerRun = 2;
+        int minRecordSize = pageSize / 10;
+        int maxRecordSize = pageSize / 8;
+
+        for (int topK = 1; topK < pageSize * numRuns * numFramesPerRun / maxRecordSize / 2; topK++) {
+            IHyracksTaskContext ctx = testUtils.create(pageSize);
+            List<Map<Integer, String>> keyValueMapList = new ArrayList<>(numRuns);
+            List<TestFrameReader> readerList = new ArrayList<>(numRuns);
+            List<IFrame> frameList = new ArrayList<>(numRuns);
+            prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+                    minRecordSize, maxRecordSize, readerList, frameList, keyValueMapList);
+
+            RunMergingFrameReader reader = new RunMergingFrameReader(ctx, readerList, frameList, SortFields,
+                    Comparators,
+                    null, RecordDesc, topK);
+            int totoalCount = testMergeSucceedInner(ctx, reader, keyValueMapList);
+            int newCount = 0;
+            for (Map<Integer, String> x : keyValueMapList) {
+                newCount += x.size();
+            }
+            assertEquals(topK + newCount, totoalCount);
+        }
+    }
+
+    private void testMergeSucceed(IHyracksTaskContext ctx, RunMergingFrameReader reader,
+            List<Map<Integer, String>> keyValueMapList) throws HyracksDataException {
+
+        testMergeSucceedInner(ctx, reader, keyValueMapList);
+        assertAllKeyValueIsConsumed(keyValueMapList);
+        reader.close();
+    }
+
+    private int testMergeSucceedInner(IHyracksTaskContext ctx, RunMergingFrameReader reader,
+            List<Map<Integer, String>> keyValueMapList) throws HyracksDataException {
+
+        IFrame frame = new VSizeFrame(ctx);
+        reader.open();
+        int count = 0;
+        for (int i = 0; i < keyValueMapList.size(); i++) {
+            keyValueMapList.set(i, new TreeMap<>(keyValueMapList.get(i)));
+            count += keyValueMapList.get(i).size();
+        }
+        while (reader.nextFrame(frame)) {
+            assertFrameIsSorted(frame, keyValueMapList);
+        }
+        return count;
+    }
+
+    @Test
+    public void testOneLargeRunMerge() throws HyracksDataException {
+        int pageSize = 64;
+        int numRuns = 2;
+        int numFramesPerRun = 1;
+        int minRecordSize = pageSize / 10;
+        int maxRecordSize = pageSize / 8;
+
+        IHyracksTaskContext ctx = testUtils.create(pageSize);
+        List<Map<Integer, String>> keyValueMap = new ArrayList<>();
+        List<TestFrameReader> readerList = new ArrayList<>();
+        List<IFrame> frameList = new ArrayList<>();
+        prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+                minRecordSize, maxRecordSize, readerList, frameList, keyValueMap);
+
+        minRecordSize = pageSize;
+        maxRecordSize = pageSize;
+        numFramesPerRun = 4;
+        prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+                minRecordSize, maxRecordSize, readerList, frameList, keyValueMap);
+
+        minRecordSize = pageSize * 2;
+        maxRecordSize = pageSize * 2;
+        numFramesPerRun = 6;
+        prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+                minRecordSize, maxRecordSize, readerList, frameList, keyValueMap);
+
+        RunMergingFrameReader reader = new RunMergingFrameReader(ctx, readerList, frameList, SortFields,
+                Comparators,
+                null,
+                RecordDesc);
+        testMergeSucceed(ctx, reader, keyValueMap);
+    }
+
+    @Test
+    public void testRunFileReader() throws HyracksDataException {
+        int pageSize = 128;
+        int numRuns = 4;
+        int numFramesPerRun = 4;
+        int minRecordSize = pageSize / 10;
+        int maxRecordSize = pageSize / 2;
+
+        IHyracksTaskContext ctx = testUtils.create(pageSize);
+        ExternalSortRunGenerator runGenerator = new ExternalSortRunGenerator(ctx, SortFields,
+                null, ComparatorFactories, RecordDesc, Algorithm.MERGE_SORT,
+                numFramesPerRun);
+
+        runGenerator.open();
+        Map<Integer, String> keyValuePair = new HashMap<>();
+        List<IFrame> frameList = new ArrayList<>();
+        prepareData(ctx, frameList, pageSize * numFramesPerRun * numRuns, minRecordSize, maxRecordSize,
+                null, keyValuePair);
+        for (IFrame frame : frameList) {
+            runGenerator.nextFrame(frame.getBuffer());
+        }
+
+        numFramesPerRun = 2;
+        minRecordSize = pageSize;
+        maxRecordSize = pageSize;
+        frameList.clear();
+        prepareData(ctx, frameList, pageSize * numFramesPerRun * numRuns, minRecordSize, maxRecordSize,
+                null, keyValuePair);
+        for (IFrame frame : frameList) {
+            runGenerator.nextFrame(frame.getBuffer());
+        }
+
+        runGenerator.close();
+        List<IFrame> inFrame = new ArrayList<>(runGenerator.getRuns().size());
+        for (RunAndMaxFrameSizePair max : runGenerator.getRuns()) {
+            inFrame.add(new GroupVSizeFrame(ctx, max.maxFrameSize));
+        }
+        matchResult(ctx, runGenerator.getRuns(), keyValuePair);
+        List<IFrameReader> runs = new ArrayList<>();
+        for (RunAndMaxFrameSizePair run : runGenerator.getRuns()) {
+            runs.add(run.run);
+        }
+        RunMergingFrameReader reader = new RunMergingFrameReader(ctx, runs, inFrame, SortFields, Comparators, null,
+                RecordDesc);
+
+        IFrame outFrame = new VSizeFrame(ctx);
+        reader.open();
+        while (reader.nextFrame(outFrame)) {
+            assertFrameIsSorted(outFrame, Arrays.asList(keyValuePair));
+        }
+        reader.close();
+        assertAllKeyValueIsConsumed(Arrays.asList(keyValuePair));
+    }
+
+    private void assertAllKeyValueIsConsumed(List<Map<Integer, String>> keyValueMapList) {
+        for (Map<Integer, String> map : keyValueMapList) {
+            assertTrue(map.isEmpty());
+        }
+    }
+
+    private void assertFrameIsSorted(IFrame frame, List<Map<Integer, String>> keyValueMapList)
+            throws HyracksDataException {
+        FrameTupleAccessor fta = new FrameTupleAccessor(RecordDesc);
+
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+        DataInputStream di = new DataInputStream(bbis);
+
+        fta.reset(frame.getBuffer());
+        //        fta.prettyPrint();
+        int preKey = Integer.MIN_VALUE;
+        for (int i = 0; i < fta.getTupleCount(); i++) {
+            bbis.setByteBuffer(fta.getBuffer(),
+                    fta.getTupleStartOffset(i) + fta.getFieldStartOffset(i, 0) + fta.getFieldSlotsLength());
+            int key = (int) RecordDesc.getFields()[0].deserialize(di);
+            bbis.setByteBuffer(fta.getBuffer(),
+                    fta.getTupleStartOffset(i) + fta.getFieldStartOffset(i, 1) + fta.getFieldSlotsLength());
+            String value = (String) RecordDesc.getFields()[1].deserialize(di);
+
+            boolean found = false;
+            for (Map<Integer, String> map : keyValueMapList) {
+                if (map.containsKey(key) && map.get(key).equals(value)) {
+                    found = true;
+                    map.remove(key);
+                    break;
+                }
+            }
+            assertTrue(found);
+            assertTrue(preKey <= key);
+            preKey = key;
+        }
+    }
+
+    static void prepareRandomInputRunList(IHyracksTaskContext ctx, int pageSize, int numRuns,
+            int numFramesPerRun, int minRecordSize, int maxRecordSize,
+            List<TestFrameReader> readerList, List<IFrame> frameList, List<Map<Integer, String>> keyValueMap)
+            throws HyracksDataException {
+        for (int i = 0; i < numRuns; i++) {
+            readerList.add(new TestFrameReader(pageSize, numFramesPerRun, minRecordSize, maxRecordSize));
+            frameList.add(new VSizeFrame(ctx, readerList.get(readerList.size() - 1).maxFrameSize));
+            keyValueMap.add(readerList.get(readerList.size() - 1).result);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/TopKRunGeneratorTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/TopKRunGeneratorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/TopKRunGeneratorTest.java
new file mode 100644
index 0000000..ae0397b
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/TopKRunGeneratorTest.java
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import static edu.uci.ics.hyracks.tests.unit.AbstractRunGeneratorTest.ComparatorFactories;
+import static edu.uci.ics.hyracks.tests.unit.AbstractRunGeneratorTest.RecordDesc;
+import static edu.uci.ics.hyracks.tests.unit.AbstractRunGeneratorTest.SerDers;
+import static edu.uci.ics.hyracks.tests.unit.AbstractRunGeneratorTest.SortFields;
+import static edu.uci.ics.hyracks.tests.unit.AbstractRunGeneratorTest.assertFTADataIsSorted;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.FixedSizeFrame;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+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.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.HybridTopKSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.HeapSortRunGenerator;
+
+public class TopKRunGeneratorTest {
+
+    static final int PAGE_SIZE = 512;
+    static final int NUM_PAGES = 80;
+    static final int SORT_FRAME_LIMIT = 4;
+
+    enum ORDER {
+        INORDER,
+        REVERSE
+    }
+
+    public class InMemorySortDataValidator implements IFrameWriter {
+
+        InMemorySortDataValidator(Map<Integer, String> answer) {
+            this.answer = answer;
+        }
+
+        Map<Integer, String> answer;
+        FrameTupleAccessor accessor;
+        int preKey = Integer.MIN_VALUE;
+
+        @Override
+        public void open() throws HyracksDataException {
+            accessor = new FrameTupleAccessor(RecordDesc);
+            preKey = Integer.MIN_VALUE;
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            accessor.reset(buffer);
+            preKey = assertFTADataIsSorted(accessor, answer, preKey);
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            assertTrue(answer.isEmpty());
+        }
+    }
+
+    @Test
+    public void testReverseOrderedDataShouldNotGenerateAnyRuns() throws HyracksDataException {
+        int topK = 1;
+        IHyracksTaskContext ctx = AbstractRunGeneratorTest.testUtils.create(PAGE_SIZE);
+        HeapSortRunGenerator sorter = new HeapSortRunGenerator(ctx, SORT_FRAME_LIMIT, topK,
+                SortFields, null, ComparatorFactories, RecordDesc);
+
+        testInMemoryOnly(ctx, topK, ORDER.REVERSE, sorter);
+    }
+
+    @Test
+    public void testAlreadySortedDataShouldNotGenerateAnyRuns() throws HyracksDataException {
+        int topK = SORT_FRAME_LIMIT;
+        IHyracksTaskContext ctx = AbstractRunGeneratorTest.testUtils.create(PAGE_SIZE);
+        HeapSortRunGenerator sorter = new HeapSortRunGenerator(ctx, SORT_FRAME_LIMIT, topK,
+                SortFields, null, ComparatorFactories, RecordDesc);
+
+        testInMemoryOnly(ctx, topK, ORDER.INORDER, sorter);
+    }
+
+    @Test
+    public void testHybridTopKShouldNotGenerateAnyRuns() throws HyracksDataException {
+        int topK = 1;
+        IHyracksTaskContext ctx = AbstractRunGeneratorTest.testUtils.create(PAGE_SIZE);
+        AbstractSortRunGenerator sorter = new HybridTopKSortRunGenerator(ctx, SORT_FRAME_LIMIT, topK,
+                SortFields, null, ComparatorFactories, RecordDesc);
+
+        testInMemoryOnly(ctx, topK, ORDER.REVERSE, sorter);
+    }
+
+    @Test
+    public void testHybridTopKShouldSwitchToFrameSorterWhenFlushed() {
+        int topK = 1;
+        IHyracksTaskContext ctx = AbstractRunGeneratorTest.testUtils.create(PAGE_SIZE);
+        AbstractSortRunGenerator sorter = new HybridTopKSortRunGenerator(ctx, SORT_FRAME_LIMIT, topK,
+                SortFields, null, ComparatorFactories, RecordDesc);
+
+    }
+
+    private void testInMemoryOnly(IHyracksTaskContext ctx, int topK, ORDER order, AbstractSortRunGenerator sorter)
+            throws HyracksDataException {
+        Map<Integer, String> keyValuePair = null;
+        switch (order) {
+            case INORDER:
+                keyValuePair = new TreeMap<>();
+                break;
+            case REVERSE:
+                keyValuePair = new TreeMap<>(Collections.reverseOrder());
+                break;
+        }
+
+        List<IFrame> frameList = new ArrayList<>();
+        int minDataSize = PAGE_SIZE * NUM_PAGES * 4 / 5;
+        int minRecordSize = 16;
+        int maxRecordSize = 64;
+
+        AbstractRunGeneratorTest
+                .prepareData(ctx, frameList, minDataSize, minRecordSize, maxRecordSize, null, keyValuePair);
+
+        assert topK > 0;
+
+        ByteBuffer buffer = prepareSortedData(keyValuePair);
+
+        Map<Integer, String> topKAnswer = getTopKAnswer(keyValuePair, topK);
+
+        doSort(sorter, buffer);
+
+        assertEquals(0, sorter.getRuns().size());
+        validateResult(sorter, topKAnswer);
+    }
+
+    private void validateResult(AbstractSortRunGenerator sorter, Map<Integer, String> topKAnswer)
+            throws HyracksDataException {
+
+        InMemorySortDataValidator validator = new InMemorySortDataValidator(topKAnswer);
+        validator.open();
+        sorter.getSorter().flush(validator);
+        validator.close();
+    }
+
+    private void doSort(AbstractSortRunGenerator sorter, ByteBuffer buffer) throws HyracksDataException {
+
+        sorter.open();
+        sorter.nextFrame(buffer);
+        sorter.close();
+    }
+
+    private Map<Integer, String> getTopKAnswer(Map<Integer, String> keyValuePair, int topK) {
+
+        TreeMap<Integer, String> copy = new TreeMap<>(keyValuePair);
+
+        Map<Integer, String> answer = new TreeMap<>();
+        for (Map.Entry<Integer, String> entry : copy.entrySet()) {
+            if (answer.size() < topK) {
+                answer.put(entry.getKey(), entry.getValue());
+            } else {
+                break;
+            }
+        }
+        return answer;
+    }
+
+    private ByteBuffer prepareSortedData(Map<Integer, String> keyValuePair) throws HyracksDataException {
+        ByteBuffer buffer = ByteBuffer.allocate(PAGE_SIZE * NUM_PAGES);
+        IFrame inputFrame = new FixedSizeFrame(buffer);
+        FrameTupleAppender appender = new FrameTupleAppender();
+        appender.reset(inputFrame, true);
+        ArrayTupleBuilder builder = new ArrayTupleBuilder(RecordDesc.getFieldCount());
+
+        for (Map.Entry<Integer, String> entry : keyValuePair.entrySet()) {
+            builder.reset();
+            builder.addField(SerDers[0], entry.getKey());
+            builder.addField(SerDers[1], entry.getValue());
+            appender.append(builder.getFieldEndOffsets(), builder.getByteArray(), 0, builder.getSize());
+        }
+        return buffer;
+    }
+}


[05/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TupleSorterHeapSort.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
new file mode 100644
index 0000000..8f8518c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
@@ -0,0 +1,269 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.ITupleBufferAccessor;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.ITupleBufferManager;
+import edu.uci.ics.hyracks.dataflow.std.structures.IResetableComparable;
+import edu.uci.ics.hyracks.dataflow.std.structures.IResetableComparableFactory;
+import edu.uci.ics.hyracks.dataflow.std.structures.MaxHeap;
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public class TupleSorterHeapSort implements ITupleSorter {
+
+    private static final Logger LOGGER = Logger.getLogger(TupleSorterHeapSort.class.getName());
+
+    class HeapEntryFactory implements IResetableComparableFactory<HeapEntry> {
+        @Override
+        public IResetableComparable<HeapEntry> createResetableComparable() {
+            return new HeapEntry();
+        }
+    }
+
+    class HeapEntry implements IResetableComparable<HeapEntry> {
+        int nmk;
+        TuplePointer tuplePointer;
+
+        public HeapEntry() {
+            tuplePointer = new TuplePointer();
+            nmk = 0;
+        }
+
+        @Override
+        public int compareTo(HeapEntry o) {
+            if (nmk != o.nmk) {
+                return ((((long) nmk) & 0xffffffffL) < (((long) o.nmk) & 0xffffffffL)) ? -1 : 1;
+            }
+            bufferAccessor1.reset(tuplePointer);
+            bufferAccessor2.reset(o.tuplePointer);
+            byte[] b1 = bufferAccessor1.getTupleBuffer().array();
+            byte[] b2 = bufferAccessor2.getTupleBuffer().array();
+
+            for (int f = 0; f < comparators.length; ++f) {
+                int fIdx = sortFields[f];
+                int s1 = bufferAccessor1.getAbsFieldStartOffset(fIdx);
+                int l1 = bufferAccessor1.getFieldLength(fIdx);
+
+                int s2 = bufferAccessor2.getAbsFieldStartOffset(fIdx);
+                int l2 = bufferAccessor2.getFieldLength(fIdx);
+                int c;
+                try {
+                    c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                } catch (HyracksDataException e) {
+                    throw new IllegalStateException(e);
+                }
+                if (c != 0) {
+                    return c;
+                }
+            }
+            return 0;
+        }
+
+        public void reset(int nmkey) {
+            nmk = nmkey;
+        }
+
+        @Override
+        public void reset(HeapEntry other) {
+            nmk = other.nmk;
+            tuplePointer.reset(other.tuplePointer);
+        }
+    }
+
+    private final ITupleBufferManager bufferManager;
+    private final ITupleBufferAccessor bufferAccessor1;
+    private final ITupleBufferAccessor bufferAccessor2;
+    private final int topK;
+    private final FrameTupleAppender outputAppender;
+    private final IFrame outputFrame;
+    private final int[] sortFields;
+    private final INormalizedKeyComputer nkc;
+    private final IBinaryComparator[] comparators;
+
+    private HeapEntry maxEntry;
+    private HeapEntry newEntry;
+
+    private MaxHeap heap;
+    private boolean isSorted;
+
+    public TupleSorterHeapSort(IHyracksTaskContext ctx, ITupleBufferManager bufferManager, int topK, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories)
+            throws HyracksDataException {
+        this.bufferManager = bufferManager;
+        this.bufferAccessor1 = bufferManager.getTupleAccessor();
+        this.bufferAccessor2 = bufferManager.getTupleAccessor();
+        this.topK = topK;
+        this.outputFrame = new VSizeFrame(ctx);
+        this.outputAppender = new FrameTupleAppender();
+        this.sortFields = sortFields;
+        this.nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
+        this.comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+
+        this.heap = new MaxHeap(new HeapEntryFactory(), topK);
+        this.maxEntry = new HeapEntry();
+        this.newEntry = new HeapEntry();
+        this.isSorted = false;
+    }
+
+    @Override
+    public int getTupleCount() {
+        return heap.getNumEntries();
+    }
+
+    @Override
+    public boolean insertTuple(IFrameTupleAccessor frameTupleAccessor, int index) throws HyracksDataException {
+        if (isSorted) {
+            throw new HyracksDataException(
+                    "The Heap haven't be reset after sorting, the order of using this class is not correct.");
+        }
+        int nmkey = getPNK(frameTupleAccessor, index);
+        if (heap.getNumEntries() >= topK) {
+            heap.peekMax(maxEntry);
+            if (compareTuple(frameTupleAccessor, index, nmkey, maxEntry) >= 0) {
+                return true;
+            }
+        }
+
+        newEntry.reset(nmkey);
+        if (!bufferManager.insertTuple(frameTupleAccessor, index, newEntry.tuplePointer)) {
+            return false;
+        }
+        if (heap.getNumEntries() < topK) {
+            heap.insert(newEntry);
+        } else {
+            bufferManager.deleteTuple(maxEntry.tuplePointer);
+            heap.replaceMax(newEntry);
+        }
+        return true;
+    }
+
+    private int getPNK(IFrameTupleAccessor fta, int tIx) {
+        if (nkc == null) {
+            return 0;
+        }
+        int sfIdx = sortFields[0];
+        return nkc.normalize(fta.getBuffer().array(), fta.getAbsoluteFieldStartOffset(tIx, sfIdx),
+                fta.getFieldLength(tIx, sfIdx));
+    }
+
+    private int compareTuple(IFrameTupleAccessor frameTupleAccessor, int tid, int nmkey, HeapEntry maxEntry)
+            throws HyracksDataException {
+        if (nmkey != maxEntry.nmk) {
+            return ((((long) nmkey) & 0xffffffffL) < (((long) maxEntry.nmk) & 0xffffffffL)) ? -1 : 1;
+        }
+        bufferAccessor2.reset(maxEntry.tuplePointer);
+        byte[] b1 = frameTupleAccessor.getBuffer().array();
+        byte[] b2 = bufferAccessor2.getTupleBuffer().array();
+
+        for (int f = 0; f < comparators.length; ++f) {
+            int fIdx = sortFields[f];
+            int s1 = frameTupleAccessor.getAbsoluteFieldStartOffset(tid, fIdx);
+            int l1 = frameTupleAccessor.getFieldLength(tid, fIdx);
+
+            int s2 = bufferAccessor2.getAbsFieldStartOffset(fIdx);
+            int l2 = bufferAccessor2.getFieldLength(fIdx);
+            int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+
+    @Override
+    public boolean hasRemaining() {
+        return getTupleCount() > 0;
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        bufferManager.reset();
+        heap.reset();
+        isSorted = false;
+    }
+
+    @Override
+    @SuppressWarnings("deprecation")
+    public void sort() throws HyracksDataException {
+        IResetableComparable[] entries = heap.getEntries();
+        int count = heap.getNumEntries();
+        Arrays.sort(entries, 0, count, entryComparator);
+        isSorted = true;
+    }
+
+    private static final Comparator<IResetableComparable> entryComparator = new Comparator<IResetableComparable>() {
+        @Override
+        public int compare(IResetableComparable o1, IResetableComparable o2) {
+            return o1.compareTo(o2);
+        }
+    };
+
+    @Override
+    public void close() {
+        heap = null;
+        bufferManager.close();
+        isSorted = false;
+    }
+
+    @Override
+    @SuppressWarnings("deprecation")
+    public int flush(IFrameWriter writer) throws HyracksDataException {
+        outputAppender.reset(outputFrame, true);
+        int maxFrameSize = outputFrame.getFrameSize();
+        int numEntries = heap.getNumEntries();
+        IResetableComparable[] entries = heap.getEntries();
+        int io = 0;
+        for (int i = 0; i < numEntries; i++) {
+            HeapEntry minEntry = (HeapEntry) entries[i];
+            bufferAccessor1.reset(minEntry.tuplePointer);
+            int flushed = FrameUtils
+                    .appendToWriter(writer, outputAppender, bufferAccessor1.getTupleBuffer().array(),
+                            bufferAccessor1.getTupleStartOffset(), bufferAccessor1.getTupleLength());
+            if (flushed > 0) {
+                maxFrameSize = Math.max(maxFrameSize, flushed);
+                io++;
+            }
+        }
+        maxFrameSize = Math.max(maxFrameSize, outputFrame.getFrameSize());
+        outputAppender.flush(writer, true);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(
+                    "Flushed records:" + numEntries + "; Flushed through " + (io + 1) + " frames");
+        }
+        return maxFrameSize;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
new file mode 100644
index 0000000..26da494
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+public enum EnumFreeSlotPolicy {
+    SMALLEST_FIT,
+    LAST_FIT,
+    BIGGEST_FIT,
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirst.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirst.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirst.java
new file mode 100644
index 0000000..085a1e8
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirst.java
@@ -0,0 +1,97 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import edu.uci.ics.hyracks.dataflow.std.structures.IResetableComparable;
+import edu.uci.ics.hyracks.dataflow.std.structures.IResetableComparableFactory;
+import edu.uci.ics.hyracks.dataflow.std.structures.MaxHeap;
+
+public class FrameFreeSlotBiggestFirst implements IFrameFreeSlotPolicy {
+    private static final int INVALID = -1;
+
+    class SpaceEntryFactory implements IResetableComparableFactory {
+        @Override
+        public IResetableComparable createResetableComparable() {
+            return new SpaceEntry();
+        }
+    }
+
+    class SpaceEntry implements IResetableComparable<SpaceEntry> {
+        int space;
+        int id;
+
+        SpaceEntry() {
+            space = INVALID;
+            id = INVALID;
+        }
+
+        @Override
+        public int compareTo(SpaceEntry o) {
+            if (o.space != space) {
+                if (o.space == INVALID) {
+                    return 1;
+                }
+                if (space == INVALID) {
+                    return -1;
+                }
+                return space < o.space ? -1 : 1;
+            }
+            return 0;
+        }
+
+        @Override
+        public void reset(SpaceEntry other) {
+            space = other.space;
+            id = other.id;
+        }
+
+        void reset(int space, int id) {
+            this.space = space;
+            this.id = id;
+        }
+    }
+
+    private MaxHeap heap;
+    private SpaceEntry tempEntry;
+
+    public FrameFreeSlotBiggestFirst(int initialCapacity) {
+        heap = new MaxHeap(new SpaceEntryFactory(), initialCapacity);
+        tempEntry = new SpaceEntry();
+    }
+
+    @Override
+    public int popBestFit(int tobeInsertedSize) {
+        if (!heap.isEmpty()) {
+            heap.peekMax(tempEntry);
+            if (tempEntry.space >= tobeInsertedSize) {
+                heap.getMax(tempEntry);
+                return tempEntry.id;
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public void pushNewFrame(int frameID, int freeSpace) {
+        tempEntry.reset(freeSpace, frameID);
+        heap.insert(tempEntry);
+    }
+
+    @Override
+    public void reset() {
+        heap.reset();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFit.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFit.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFit.java
new file mode 100644
index 0000000..0bfcf38
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFit.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.util.Arrays;
+
+public class FrameFreeSlotLastFit implements IFrameFreeSlotPolicy {
+    private static int INITIAL_CAPACITY = 10;
+
+    private class FrameSpace {
+        int frameId;
+        int freeSpace;
+
+        FrameSpace(int frameId, int freeSpace) {
+            reset(frameId, freeSpace);
+        }
+
+        void reset(int frameId, int freeSpace) {
+            this.frameId = frameId;
+            this.freeSpace = freeSpace;
+        }
+    }
+
+    private FrameSpace[] frameSpaces;
+    private int size;
+
+    public FrameFreeSlotLastFit(int maxFrames) {
+        frameSpaces = new FrameSpace[maxFrames];
+        size = 0;
+    }
+
+    public FrameFreeSlotLastFit() {
+        this(INITIAL_CAPACITY);
+    }
+
+    @Override
+    public int popBestFit(int tobeInsertedSize) {
+        for (int i = size - 1; i >= 0; i--) {
+            if (frameSpaces[i].freeSpace >= tobeInsertedSize) {
+                FrameSpace ret = frameSpaces[i];
+                System.arraycopy(frameSpaces, i + 1, frameSpaces, i, size - i - 1);
+                frameSpaces[--size] = ret;
+                return ret.frameId;
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public void pushNewFrame(int frameID, int freeSpace) {
+        if (size >= frameSpaces.length) {
+            frameSpaces = Arrays.copyOf(frameSpaces, size * 2);
+        }
+        if (frameSpaces[size] == null) {
+            frameSpaces[size++] = new FrameSpace(frameID, freeSpace);
+        } else {
+            frameSpaces[size++].reset(frameID, freeSpace);
+        }
+    }
+
+    @Override
+    public void reset() {
+        size = 0;
+        for (int i = frameSpaces.length - 1; i >= 0; i--) {
+            frameSpaces[i] = null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotSmallestFit.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotSmallestFit.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotSmallestFit.java
new file mode 100644
index 0000000..69e1911
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotSmallestFit.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.TreeMap;
+
+public class FrameFreeSlotSmallestFit implements IFrameFreeSlotPolicy {
+
+    private TreeMap<Integer, LinkedList<Integer>> freeSpaceIndex;
+
+    public FrameFreeSlotSmallestFit() {
+        freeSpaceIndex = new TreeMap<>();
+    }
+
+    @Override
+    public int popBestFit(int tobeInsertedSize) {
+        Map.Entry<Integer, LinkedList<Integer>> entry = freeSpaceIndex.ceilingEntry(tobeInsertedSize);
+        if (entry == null) {
+            return -1;
+        }
+        int id = entry.getValue().removeFirst();
+        if (entry.getValue().isEmpty()) {
+            freeSpaceIndex.remove(entry.getKey());
+        }
+        return id;
+    }
+
+    @Override
+    public void pushNewFrame(int frameID, int freeSpace) {
+        Map.Entry<Integer, LinkedList<Integer>> entry = freeSpaceIndex.ceilingEntry(freeSpace);
+        if (entry == null || entry.getKey() != freeSpace) {
+            LinkedList<Integer> linkedList = new LinkedList<>();
+            linkedList.add(frameID);
+            freeSpaceIndex.put(freeSpace, linkedList);
+        } else {
+            entry.getValue().add(frameID);
+        }
+    }
+
+    @Override
+    public void reset() {
+        freeSpaceIndex.clear();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameBufferManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameBufferManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameBufferManager.java
new file mode 100644
index 0000000..9a52efa
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameBufferManager.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameBufferManager {
+
+    /**
+     * Reset the counters and flags to initial status. This method should not release the pre-allocated resources
+     *
+     * @throws edu.uci.ics.hyracks.api.exceptions.HyracksDataException
+     */
+    void reset() throws HyracksDataException;
+
+    /**
+     * @param frameIndex
+     * @return the specified frame, from the set of memory buffers, being
+     * managed by this memory manager
+     */
+    ByteBuffer getFrame(int frameIndex);
+
+    /**
+     * Get the startOffset of the specific frame inside buffer
+     *
+     * @param frameIndex
+     * @return the start offset of the frame returned by {@link #getFrame(int)} method.
+     */
+    int getFrameStartOffset(int frameIndex);
+
+    /**
+     * Get the size of the specific frame inside buffer
+     *
+     * @param frameIndex
+     * @return the length of the specific frame
+     */
+    int getFrameSize(int frameIndex);
+
+    /**
+     * @return the number of frames in this buffer
+     */
+    int getNumFrames();
+
+    /**
+     * Writes the whole frame into the buffer.
+     *
+     * @param frame source frame
+     * @return the id of the inserted frame. if failed to return it will be -1.
+     */
+    int insertFrame(ByteBuffer frame) throws HyracksDataException;
+
+    void close();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameFreeSlotPolicy.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameFreeSlotPolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameFreeSlotPolicy.java
new file mode 100644
index 0000000..57a8094
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameFreeSlotPolicy.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+public interface IFrameFreeSlotPolicy {
+
+    /**
+     * Find the best fit frame id which can hold the data, and then pop it out from the index.
+     * Return -1 is failed to find any.
+     *
+     * @param tobeInsertedSize the actual size of the data which should include
+     *                         the meta data like the field offset and the tuple
+     *                         count extra size
+     * @return the best fit frame id
+     */
+    int popBestFit(int tobeInsertedSize);
+
+    /**
+     * Register the new free slot into the index
+     *
+     * @param frameID
+     * @param freeSpace
+     */
+    void pushNewFrame(int frameID, int freeSpace);
+
+    /**
+     * Clear all the existing free slot information.
+     */
+    void reset();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFramePool.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFramePool.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFramePool.java
new file mode 100644
index 0000000..1e5be25
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFramePool.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFramePool {
+
+    int getMinFrameSize();
+
+    int getMemoryBudgetBytes();
+
+    /**
+     * Get a frame of given size. <br>
+     * Returns {@code null} if failed to allocate the required size of frame
+     *
+     * @param frameSize the actual size of the frame.
+     * @return the allocated frame
+     * @throws HyracksDataException
+     */
+    ByteBuffer allocateFrame(int frameSize) throws HyracksDataException;
+
+    /**
+     * Reset the counters to initial status. This method should not release the pre-allocated resources.
+     */
+    void reset();
+
+    /**
+     * Release the pre-allocated resources.
+     */
+    void close();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java
new file mode 100644
index 0000000..49a664c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public interface ITupleBufferAccessor {
+
+    void reset(TuplePointer tuplePointer);
+
+    ByteBuffer getTupleBuffer();
+
+    int getTupleStartOffset();
+
+    int getTupleLength();
+
+    int getAbsFieldStartOffset(int fieldId);
+
+    int getFieldLength(int fieldId);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
new file mode 100644
index 0000000..6f94563
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public interface ITupleBufferManager {
+    /**
+     * Reset the counters and flags to initial status. This method should not release the pre-allocated resources
+     *
+     * @throws edu.uci.ics.hyracks.api.exceptions.HyracksDataException
+     */
+    void reset() throws HyracksDataException;
+
+    /**
+     * @return the number of tuples in this buffer
+     */
+    int getNumTuples();
+
+    boolean insertTuple(IFrameTupleAccessor accessor, int idx, TuplePointer tuplePointer) throws HyracksDataException;
+
+    void deleteTuple(TuplePointer tuplePointer) throws HyracksDataException;
+
+    void close();
+
+    ITupleBufferAccessor getTupleAccessor();
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFrameMemoryManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFrameMemoryManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFrameMemoryManager.java
new file mode 100644
index 0000000..834ba2a
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFrameMemoryManager.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *  
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *  
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.FixedSizeFrame;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+
+public class VariableFrameMemoryManager implements IFrameBufferManager {
+
+    private class PhysicalFrameOffset {
+        IFrame physicalFrame;
+        int physicalOffset;
+
+        PhysicalFrameOffset(IFrame frame, int offset) {
+            physicalFrame = frame;
+            physicalOffset = offset;
+        }
+    }
+
+    private class LogicalFrameStartSize {
+        ByteBuffer logicalFrame;
+        int logicalStart;
+        int logicalSize;
+
+        LogicalFrameStartSize(ByteBuffer frame, int start, int size) {
+            logicalFrame = frame;
+            logicalStart = start;
+            logicalSize = size;
+        }
+    }
+
+    private final IFramePool framePool;
+    private List<PhysicalFrameOffset> physicalFrameOffsets;
+    private List<LogicalFrameStartSize> logicalFrameStartSizes;
+    private final IFrameFreeSlotPolicy freeSlotPolicy;
+
+    public VariableFrameMemoryManager(IFramePool framePool, IFrameFreeSlotPolicy freeSlotPolicy) {
+        this.framePool = framePool;
+        this.freeSlotPolicy = freeSlotPolicy;
+        int maxFrames = framePool.getMemoryBudgetBytes() / framePool.getMinFrameSize();
+        this.physicalFrameOffsets = new ArrayList<>(maxFrames);
+        this.logicalFrameStartSizes = new ArrayList<>(maxFrames);
+    }
+
+    private int findAvailableFrame(int frameSize) throws HyracksDataException {
+        int frameId = freeSlotPolicy.popBestFit(frameSize);
+        if (frameId >= 0) {
+            return frameId;
+        }
+        ByteBuffer buffer = framePool.allocateFrame(frameSize);
+        if (buffer != null) {
+            IntSerDeUtils.putInt(buffer.array(), FrameHelper.getTupleCountOffset(buffer.capacity()), 0);
+            physicalFrameOffsets.add(new PhysicalFrameOffset(new FixedSizeFrame(buffer), 0));
+            return physicalFrameOffsets.size() - 1;
+        }
+        return -1;
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        physicalFrameOffsets.clear();
+        logicalFrameStartSizes.clear();
+        freeSlotPolicy.reset();
+        framePool.reset();
+    }
+
+    @Override
+    public ByteBuffer getFrame(int frameIndex) {
+        return logicalFrameStartSizes.get(frameIndex).logicalFrame;
+    }
+
+    @Override
+    public int getFrameStartOffset(int frameIndex) {
+        return logicalFrameStartSizes.get(frameIndex).logicalStart;
+    }
+
+    @Override
+    public int getFrameSize(int frameIndex) {
+        return logicalFrameStartSizes.get(frameIndex).logicalSize;
+    }
+
+    @Override
+    public int getNumFrames() {
+        return logicalFrameStartSizes.size();
+    }
+
+    @Override
+    public int insertFrame(ByteBuffer frame) throws HyracksDataException {
+        int frameSize = frame.capacity();
+        int physicalFrameId = findAvailableFrame(frameSize);
+        if (physicalFrameId < 0) {
+            return -1;
+        }
+        ByteBuffer buffer = physicalFrameOffsets.get(physicalFrameId).physicalFrame.getBuffer();
+        int offset = physicalFrameOffsets.get(physicalFrameId).physicalOffset;
+        System.arraycopy(frame.array(), 0, buffer.array(), offset, frameSize);
+        if (offset + frameSize < buffer.capacity()) {
+            freeSlotPolicy.pushNewFrame(physicalFrameId, buffer.capacity() - offset - frameSize);
+        }
+        physicalFrameOffsets.get(physicalFrameId).physicalOffset = offset + frameSize;
+        logicalFrameStartSizes.add(new LogicalFrameStartSize(buffer, offset, frameSize));
+        return logicalFrameStartSizes.size() - 1;
+    }
+
+    @Override
+    public void close() {
+        physicalFrameOffsets.clear();
+        logicalFrameStartSizes.clear();
+        freeSlotPolicy.reset();
+        framePool.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePool.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePool.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePool.java
new file mode 100644
index 0000000..0d936b4
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePool.java
@@ -0,0 +1,200 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.context.IHyracksFrameMgrContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class VariableFramePool implements IFramePool {
+    public static final int UNLIMITED_MEMORY = -1;
+
+    private final IHyracksFrameMgrContext ctx;
+    private final int minFrameSize;
+    private final int memBudget;
+
+    private int allocateMem;
+    private ArrayList<ByteBuffer> buffers;  // the unused slots were sorted by size increasingly.
+    private BitSet used; // the merged one also marked as used.
+
+    /**
+     * The constructor of the VariableFramePool.
+     *
+     * @param ctx
+     * @param memBudgetInBytes the given memory budgets to allocate the frames. If it less than 0, it will be treated as unlimited budgets
+     */
+    public VariableFramePool(IHyracksFrameMgrContext ctx, int memBudgetInBytes) {
+        this.ctx = ctx;
+        this.minFrameSize = ctx.getInitialFrameSize();
+        this.allocateMem = 0;
+        if (memBudgetInBytes == UNLIMITED_MEMORY) {
+            this.memBudget = Integer.MAX_VALUE;
+            this.buffers = new ArrayList<>();
+            this.used = new BitSet();
+        } else {
+            this.memBudget = memBudgetInBytes;
+            this.buffers = new ArrayList<>(memBudgetInBytes / minFrameSize);
+            this.used = new BitSet(memBudgetInBytes / minFrameSize);
+        }
+    }
+
+    @Override
+    public int getMinFrameSize() {
+        return minFrameSize;
+    }
+
+    @Override
+    public int getMemoryBudgetBytes() {
+        return memBudget;
+    }
+
+    @Override
+    public ByteBuffer allocateFrame(int frameSize) throws HyracksDataException {
+        int frameId = findExistingFrame(frameSize);
+        if (frameId >= 0) {
+            return reuseFrame(frameId);
+        }
+        if (haveEnoughFreeSpace(frameSize)) {
+            return createNewFrame(frameSize);
+        }
+        return mergeExistingFrames(frameSize);
+
+    }
+
+    private boolean haveEnoughFreeSpace(int frameSize) {
+        return frameSize + allocateMem <= memBudget;
+    }
+
+    private static int getFirstUnUsedPos(BitSet used) {
+        return used.nextClearBit(0);
+    }
+
+    private static int getLastUnUsedPos(BitSet used, int lastPos) {
+        return used.previousClearBit(lastPos);
+    }
+
+    private static int binarySearchUnusedBuffer(ArrayList<ByteBuffer> buffers, BitSet used, int frameSize) {
+        int l = getFirstUnUsedPos(used); // to skip the merged null buffers
+        int h = getLastUnUsedPos(used, (buffers.size() - 1)) + 1; // to skip the newly created buffers
+        if (l >= h) {
+            return -1;
+        }
+        int highest = h;
+        int mid = (l + h) / 2;
+        while (l < h) {
+            ByteBuffer buffer = buffers.get(mid);
+            if (buffer.capacity() == frameSize) {
+                break;
+            }
+            if (buffer.capacity() < frameSize) {
+                l = mid + 1;
+            } else {
+                h = mid;
+            }
+            mid = (l + h) / 2;
+        }
+        mid = used.nextClearBit(mid);
+        return mid < highest ? mid : -1;
+    }
+
+    private int findExistingFrame(int frameSize) {
+        return binarySearchUnusedBuffer(buffers, used, frameSize);
+    }
+
+    private ByteBuffer reuseFrame(int id) {
+        used.set(id);
+        buffers.get(id).clear();
+        return buffers.get(id);
+    }
+
+    private ByteBuffer createNewFrame(int frameSize) throws HyracksDataException {
+        buffers.add(ctx.allocateFrame(frameSize));
+        allocateMem += frameSize;
+        return reuseFrame(buffers.size() - 1);
+    }
+
+    /**
+     * The merging sequence is from the smallest to the largest order.
+     * Once the buffer get merged, it will be remove from the list in order to free the object.
+     * And the index spot of it will be marked as used.
+     *
+     * @param frameSize
+     * @return
+     * @throws HyracksDataException
+     */
+    private ByteBuffer mergeExistingFrames(int frameSize) throws HyracksDataException {
+        int mergedSize = memBudget - allocateMem;
+        int highBound = getLastUnUsedPos(used, buffers.size() - 1) + 1;
+        for (int i = getFirstUnUsedPos(used); i < highBound; ++i) {
+            if (!used.get(i)) {
+                mergedSize += deAllocateFrame(i);
+                if (mergedSize >= frameSize) {
+                    return createNewFrame(mergedSize);
+                }
+            }
+        }
+        return null;
+    }
+
+    private int deAllocateFrame(int id) {
+        ByteBuffer frame = buffers.get(id);
+        ctx.deallocateFrames(frame.capacity());
+        buffers.set(id, null);
+        used.set(id);
+        allocateMem -= frame.capacity();
+        return frame.capacity();
+    }
+
+    @Override
+    public void reset() {
+        removeEmptySpot(buffers);
+        Collections.sort(buffers, sizeByteBufferComparator);
+        used.clear();
+    }
+
+    private static void removeEmptySpot(List<ByteBuffer> buffers) {
+        for (int i = 0; i < buffers.size(); ) {
+            if (buffers.get(i) == null) {
+                buffers.remove(i);
+            } else {
+                i++;
+            }
+        }
+    }
+
+    @Override
+    public void close() {
+        buffers.clear();
+        used.clear();
+        allocateMem = 0;
+    }
+
+    private static Comparator<ByteBuffer> sizeByteBufferComparator = new Comparator<ByteBuffer>() {
+        @Override
+        public int compare(ByteBuffer o1, ByteBuffer o2) {
+            if (o1.capacity() == o2.capacity()) {
+                return 0;
+            }
+            return o1.capacity() < o2.capacity() ? -1 : 1;
+        }
+    };
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
new file mode 100644
index 0000000..0b077c2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
@@ -0,0 +1,203 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.DeletableFrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.IAppendDeletableFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public class VariableTupleMemoryManager implements ITupleBufferManager {
+
+    private final static Logger LOG = Logger.getLogger(VariableTupleMemoryManager.class.getName());
+
+    private final int MIN_FREE_SPACE;
+    private final IFramePool pool;
+    private final IFrameFreeSlotPolicy policy;
+    private final IAppendDeletableFrameTupleAccessor accessor;
+    private final ArrayList<ByteBuffer> frames;
+    private final RecordDescriptor recordDescriptor;
+    private int numTuples;
+    private int statsReOrg;
+
+    public VariableTupleMemoryManager(IFramePool framePool, RecordDescriptor recordDescriptor) {
+        this.pool = framePool;
+        int maxFrames = framePool.getMemoryBudgetBytes() / framePool.getMinFrameSize();
+        this.policy = new FrameFreeSlotLastFit(maxFrames);
+        this.accessor = new DeletableFrameTupleAppender(recordDescriptor);
+        this.frames = new ArrayList<>();
+        this.MIN_FREE_SPACE = calculateMinFreeSpace(recordDescriptor);
+        this.recordDescriptor = recordDescriptor;
+        this.numTuples = 0;
+        this.statsReOrg = 0;
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        pool.reset();
+        policy.reset();
+        frames.clear();
+        numTuples = 0;
+    }
+
+    @Override
+    public int getNumTuples() {
+        return numTuples;
+    }
+
+    @Override
+    public boolean insertTuple(IFrameTupleAccessor fta, int idx, TuplePointer tuplePointer)
+            throws HyracksDataException {
+        int requiredFreeSpace = calculatePhysicalSpace(fta, idx);
+        int frameId = findAvailableFrame(requiredFreeSpace);
+        if (frameId < 0) {
+            if (canBeInsertedAfterCleanUpFragmentation(requiredFreeSpace)) {
+                reOrganizeFrames();
+                frameId = findAvailableFrame(requiredFreeSpace);
+                statsReOrg++;
+            } else {
+                return false;
+            }
+        }
+        assert frameId >= 0;
+        accessor.reset(frames.get(frameId));
+        assert accessor.getContiguousFreeSpace() >= requiredFreeSpace;
+        int tid = accessor.append(fta, idx);
+        assert tid >= 0;
+        tuplePointer.reset(frameId, tid);
+        if (accessor.getContiguousFreeSpace() > MIN_FREE_SPACE) {
+            policy.pushNewFrame(frameId, accessor.getContiguousFreeSpace());
+        }
+        numTuples++;
+        return true;
+    }
+
+    private void reOrganizeFrames() {
+        policy.reset();
+        for (int i = 0; i < frames.size(); i++) {
+            accessor.reset(frames.get(i));
+            accessor.reOrganizeBuffer();
+            policy.pushNewFrame(i, accessor.getContiguousFreeSpace());
+        }
+    }
+
+    private boolean canBeInsertedAfterCleanUpFragmentation(int requiredFreeSpace) {
+        for (int i = 0; i < frames.size(); i++) {
+            accessor.reset(frames.get(i));
+            if (accessor.getTotalFreeSpace() >= requiredFreeSpace) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    private int findAvailableFrame(int requiredFreeSpace) throws HyracksDataException {
+        int frameId = policy.popBestFit(requiredFreeSpace);
+        if (frameId >= 0) {
+            return frameId;
+        }
+
+        int frameSize = calculateMinFrameSizeToPlaceTuple(requiredFreeSpace, pool.getMinFrameSize());
+        ByteBuffer buffer = pool.allocateFrame(frameSize);
+        if (buffer != null) {
+            accessor.clear(buffer);
+            frames.add(buffer);
+            return frames.size() - 1;
+        }
+        return -1;
+    }
+
+    private static int calculateMinFrameSizeToPlaceTuple(int requiredFreeSpace, int minFrameSize) {
+        return (1 + (requiredFreeSpace + 4 - 1) / minFrameSize) * minFrameSize;
+    }
+
+    private static int calculatePhysicalSpace(IFrameTupleAccessor fta, int idx) {
+        // 4 bytes to store the offset
+        return 4 + fta.getTupleLength(idx);
+    }
+
+    private static int calculateMinFreeSpace(RecordDescriptor recordDescriptor) {
+        // + 4 for the tuple offset
+        return recordDescriptor.getFieldCount() * 4 + 4;
+    }
+
+    @Override
+    public void deleteTuple(TuplePointer tuplePointer) throws HyracksDataException {
+        accessor.reset(frames.get(tuplePointer.frameIndex));
+        accessor.delete(tuplePointer.tupleIndex);
+        numTuples--;
+    }
+
+    @Override
+    public void close() {
+        pool.close();
+        policy.reset();
+        frames.clear();
+        numTuples = 0;
+        if (LOG.isLoggable(Level.FINE)) {
+            LOG.fine("VariableTupleMemoryManager has reorganized " + statsReOrg + " times");
+        }
+        statsReOrg = 0;
+    }
+
+    @Override
+    public ITupleBufferAccessor getTupleAccessor() {
+        return new ITupleBufferAccessor() {
+            private IAppendDeletableFrameTupleAccessor bufferAccessor = new DeletableFrameTupleAppender(
+                    recordDescriptor);
+            private int tid;
+
+            @Override
+            public void reset(TuplePointer tuplePointer) {
+                bufferAccessor.reset(frames.get(tuplePointer.frameIndex));
+                tid = tuplePointer.tupleIndex;
+            }
+
+            @Override
+            public ByteBuffer getTupleBuffer() {
+                return bufferAccessor.getBuffer();
+            }
+
+            @Override
+            public int getTupleStartOffset() {
+                return bufferAccessor.getTupleStartOffset(tid);
+            }
+
+            @Override
+            public int getTupleLength() {
+                return bufferAccessor.getTupleLength(tid);
+            }
+
+            @Override
+            public int getAbsFieldStartOffset(int fieldId) {
+                return bufferAccessor.getAbsoluteFieldStartOffset(tid, fieldId);
+            }
+
+            @Override
+            public int getFieldLength(int fieldId) {
+                return bufferAccessor.getFieldLength(tid, fieldId);
+            }
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
new file mode 100644
index 0000000..9b03a77
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
@@ -0,0 +1,244 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.util;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+
+/**
+ * This is a special frame which is used in TupleMemoryBuffer.
+ * This frame has a special structure to organize the deleted spaces.
+ * Specifically, the endOffset of the deleted tuple will be set as negative number.
+ * And we add a special <code>deleted_space</code> field at the last 4 bytes to remember how many bytes has been deleted.
+ */
+public class DeletableFrameTupleAppender implements IAppendDeletableFrameTupleAccessor {
+
+    private static final int SIZE_DELETED_SPACE = 4;
+    private final RecordDescriptor recordDescriptor;
+    private ByteBuffer buffer;
+    private int tupleCountOffset;
+    private int tupleCount;
+    private int freeDataEndOffset;
+    private int deletedSpace;
+    private byte[] array;   // to speed up the array visit a little
+
+    public DeletableFrameTupleAppender(RecordDescriptor recordDescriptor) {
+        this.recordDescriptor = recordDescriptor;
+    }
+
+    private int getTupleCountOffset() {
+        return FrameHelper.getTupleCountOffset(buffer.capacity()) - SIZE_DELETED_SPACE;
+    }
+
+    private int getFreeDataEndOffset() {
+        return tupleCount == 0 ? 0 : Math.abs(IntSerDeUtils.getInt(array, tupleCountOffset - tupleCount * 4));
+    }
+
+    private void setFreeDataEndOffset(int offset) {
+        assert (offset >= 0);
+        IntSerDeUtils.putInt(array, tupleCountOffset - tupleCount * 4, offset);
+    }
+
+    private void setTupleCount(int count) {
+        IntSerDeUtils.putInt(array, tupleCountOffset, count);
+    }
+
+    private void setDeleteSpace(int count) {
+        IntSerDeUtils.putInt(array, buffer.capacity() - SIZE_DELETED_SPACE, count);
+    }
+
+    private int getPhysicalTupleCount() {
+        return IntSerDeUtils.getInt(array, tupleCountOffset);
+    }
+
+    private int getDeletedSpace() {
+        return IntSerDeUtils.getInt(array, buffer.capacity() - SIZE_DELETED_SPACE);
+    }
+
+    @Override
+    public void clear(ByteBuffer buffer) throws HyracksDataException {
+        this.buffer = buffer;
+        this.array = buffer.array();
+        tupleCountOffset = getTupleCountOffset();
+        setTupleCount(0);
+        setDeleteSpace(0);
+        resetCounts();
+    }
+
+    @Override
+    public void reset(ByteBuffer buffer) {
+        this.buffer = buffer;
+        this.array = buffer.array();
+        tupleCountOffset = getTupleCountOffset();
+        resetCounts();
+    }
+
+    private void resetCounts() {
+        deletedSpace = getDeletedSpace();
+        tupleCount = getPhysicalTupleCount();
+        freeDataEndOffset = getFreeDataEndOffset();
+    }
+
+    /**
+     * Append the record into the frame. This method will not validate the space, please make sure space is enough
+     * by calling {@link #getContiguousFreeSpace()}
+     *
+     * @param tupleAccessor
+     * @param tIndex
+     * @return
+     * @throws HyracksDataException
+     */
+    @Override
+    public int append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException {
+        byte[] src = tupleAccessor.getBuffer().array();
+        int tStartOffset = tupleAccessor.getTupleStartOffset(tIndex);
+        int length = tupleAccessor.getTupleLength(tIndex);
+        System.arraycopy(src, tStartOffset, array, freeDataEndOffset, length);
+        setTupleCount(++tupleCount);
+        freeDataEndOffset += length;
+        setFreeDataEndOffset(freeDataEndOffset);
+        return tupleCount - 1;
+    }
+
+    @Override
+    public void delete(int tupleIndex) {
+        int endOffset = getTupleEndOffset(tupleIndex);
+        if (endOffset > 0) {
+            setTupleEndOffset(tupleIndex, -endOffset);
+            deletedSpace += endOffset - getTupleStartOffset(tupleIndex);
+            setDeleteSpace(deletedSpace);
+        }
+    }
+
+    @Override
+    public void reOrganizeBuffer() {
+        if (deletedSpace <= 0) {
+            return;
+        }
+        reclaimDeletedEnding();
+
+        freeDataEndOffset = 0;
+        int endOffset = 0;
+        for (int i = 0; i < tupleCount; i++) {
+            int startOffset = Math.abs(endOffset);
+            endOffset = getTupleEndOffset(i);
+            if (endOffset >= 0) {
+                int length = endOffset - startOffset;
+                assert ( length >= 0);
+                if (freeDataEndOffset != startOffset) {
+                    System.arraycopy(array, startOffset, array, freeDataEndOffset, length);
+                }
+                freeDataEndOffset += length;
+            }
+            setTupleEndOffset(i, freeDataEndOffset);
+        }
+        setFreeDataEndOffset(freeDataEndOffset);
+        deletedSpace = 0;
+        setDeleteSpace(0);
+    }
+
+    private void reclaimDeletedEnding() {
+        for (int i = tupleCount - 1; i >= 0; i--) {
+            int endOffset = getTupleEndOffset(i);
+            if (endOffset < 0) {
+                tupleCount--;
+            } else {
+                break;
+            }
+        }
+        setTupleCount(tupleCount);
+    }
+
+    @Override
+    public int getTotalFreeSpace() {
+        return getContiguousFreeSpace() + deletedSpace;
+    }
+
+    @Override
+    public int getContiguousFreeSpace() {
+        return getTupleCountOffset() - tupleCount * 4 - freeDataEndOffset;
+    }
+
+    @Override
+    public int getFieldCount() {
+        return recordDescriptor.getFieldCount();
+    }
+
+    @Override
+    public int getFieldSlotsLength() {
+        return recordDescriptor.getFieldCount() * 4;
+    }
+
+    @Override
+    public int getFieldEndOffset(int tupleIndex, int fIdx) {
+        return IntSerDeUtils.getInt(array, getTupleStartOffset(tupleIndex) + fIdx * 4);
+    }
+
+    @Override
+    public int getFieldStartOffset(int tupleIndex, int fIdx) {
+        return fIdx == 0 ? 0 : IntSerDeUtils.getInt(array, getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+    }
+
+    @Override
+    public int getFieldLength(int tupleIndex, int fIdx) {
+        return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
+    public int getTupleLength(int tupleIndex) {
+        int endOffset = getTupleEndOffset(tupleIndex);
+        if (endOffset < 0) {
+            return endOffset + getTupleStartOffset(tupleIndex);
+        }
+        return endOffset - getTupleStartOffset(tupleIndex);
+    }
+
+    @Override
+    public int getTupleEndOffset(int tupleIndex) {
+        return IntSerDeUtils.getInt(array, tupleCountOffset - 4 * (tupleIndex + 1));
+    }
+
+    private void setTupleEndOffset(int tupleIndex, int offset) {
+        IntSerDeUtils.putInt(array, tupleCountOffset - 4 * (tupleIndex + 1), offset);
+    }
+
+    @Override
+    public int getTupleStartOffset(int tupleIndex) {
+        int offset = tupleIndex == 0 ? 0 : IntSerDeUtils.getInt(array, tupleCountOffset - 4 * tupleIndex);
+        return Math.abs(offset);
+    }
+
+    @Override
+    public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+        return getTupleStartOffset(tupleIndex) + getFieldSlotsLength() + getFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
+    public int getTupleCount() {
+        return tupleCount;
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupFrameAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupFrameAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupFrameAccessor.java
new file mode 100644
index 0000000..12ba72f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupFrameAccessor.java
@@ -0,0 +1,170 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.util;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+/**
+ * This {@code GroupFrameAccessor} access a group of logical frames which are stored in one physical
+ * continuous ByteBuffer. It is used in a RunFileReader which can read several frames at once, and we
+ * can use this accessor to parse the returned data as one frame. In the caller's view there is only
+ * one frame which simply the caller's work.
+ */
+public class GroupFrameAccessor implements IFrameTupleAccessor {
+
+    private class InnerFrameInfo implements Comparable<Integer> {
+        int start;
+        int length;
+        int tupleCount;
+
+        InnerFrameInfo(int start, int length, int tupleCount) {
+            this.start = start;
+            this.length = length;
+            this.tupleCount = tupleCount;
+        }
+
+        @Override
+        public int compareTo(Integer o) {
+            return -o.compareTo(tupleCount);
+        }
+    }
+
+    private final RecordDescriptor recordDescriptor;
+    private final int minFrameSize;
+    private final FrameTupleAccessor frameTupleAccessor;
+    private int lastTupleIndex;
+    private int lastFrameId;
+    private ByteBuffer buffer;
+    private List<InnerFrameInfo> innerFrameInfos;
+
+    public GroupFrameAccessor(int minFrameSize, RecordDescriptor recordDescriptor) {
+        this.minFrameSize = minFrameSize;
+        this.recordDescriptor = (recordDescriptor);
+        this.frameTupleAccessor = new FrameTupleAccessor(recordDescriptor);
+        this.innerFrameInfos = new ArrayList<>();
+    }
+
+    @Override
+    public int getFieldCount() {
+        return recordDescriptor.getFieldCount();
+    }
+
+    @Override
+    public int getFieldSlotsLength() {
+        return frameTupleAccessor.getFieldSlotsLength();
+    }
+
+    @Override
+    public int getFieldEndOffset(int tupleIndex, int fIdx) {
+        return frameTupleAccessor.getFieldEndOffset(resetSubTupleAccessor(tupleIndex), fIdx);
+    }
+
+    @Override
+    public int getFieldStartOffset(int tupleIndex, int fIdx) {
+        return frameTupleAccessor.getFieldStartOffset(resetSubTupleAccessor(tupleIndex), fIdx);
+    }
+
+    @Override
+    public int getFieldLength(int tupleIndex, int fIdx) {
+        return frameTupleAccessor.getFieldLength(resetSubTupleAccessor(tupleIndex), fIdx);
+    }
+
+    @Override
+    public int getTupleLength(int tupleIndex) {
+        return frameTupleAccessor.getTupleLength(resetSubTupleAccessor(tupleIndex));
+    }
+
+    @Override
+    public int getTupleEndOffset(int tupleIndex) {
+        return frameTupleAccessor.getTupleEndOffset(resetSubTupleAccessor(tupleIndex));
+    }
+
+    @Override
+    public int getTupleStartOffset(int tupleIndex) {
+        return frameTupleAccessor.getTupleStartOffset(resetSubTupleAccessor(tupleIndex));
+    }
+
+    @Override
+    public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+        return frameTupleAccessor.getAbsoluteFieldStartOffset(resetSubTupleAccessor(tupleIndex), fIdx);
+    }
+
+    @Override
+    public int getTupleCount() {
+        return innerFrameInfos.size() > 0 ? innerFrameInfos.get(innerFrameInfos.size() - 1).tupleCount : 0;
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    @Override
+    public void reset(ByteBuffer buffer) {
+        this.buffer = buffer;
+        this.lastTupleIndex = -1;
+        this.lastFrameId = -1;
+        parseGroupedBuffer(0, buffer.limit());
+    }
+
+    private void parseGroupedBuffer(int start, int stop) {
+        this.innerFrameInfos.clear();
+        int i = start;
+        while (i < stop) {
+            int unitSize = FrameHelper.deserializeNumOfMinFrame(buffer, i) * minFrameSize;
+            if (unitSize == 0) { // run consumed.
+                break;
+            }
+            if (i + unitSize > stop) { // contains future partial run, stop here
+                break;
+            }
+            frameTupleAccessor.reset(buffer, i, unitSize);
+            this.innerFrameInfos
+                    .add(new InnerFrameInfo(i, unitSize, getTupleCount() + frameTupleAccessor.getTupleCount()));
+            i += unitSize;
+        }
+        buffer.position(i); // reading stops here.
+    }
+
+    private int resetSubTupleAccessor(int tupleIndex) {
+        assert tupleIndex < getTupleCount();
+        if (innerFrameInfos.size() == 1) {
+            return tupleIndex;
+        }
+        if (tupleIndex == lastTupleIndex) {
+            return lastFrameId > 0 ? lastTupleIndex - innerFrameInfos.get(lastFrameId - 1).tupleCount : lastTupleIndex;
+        }
+        int subFrameId = Collections.binarySearch(innerFrameInfos, tupleIndex);
+        if (subFrameId >= 0) {
+            subFrameId++;
+        } else {
+            subFrameId = -subFrameId - 1;
+        }
+        frameTupleAccessor.reset(buffer, innerFrameInfos.get(subFrameId).start, innerFrameInfos.get(subFrameId).length);
+        lastTupleIndex = tupleIndex;
+        lastFrameId = subFrameId;
+        return lastFrameId > 0 ? lastTupleIndex - innerFrameInfos.get(lastFrameId - 1).tupleCount : lastTupleIndex;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupVSizeFrame.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupVSizeFrame.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupVSizeFrame.java
new file mode 100644
index 0000000..b273f5b
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupVSizeFrame.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.util;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class GroupVSizeFrame extends VSizeFrame {
+
+    public GroupVSizeFrame(IHyracksCommonContext ctx, int frameSize)
+            throws HyracksDataException {
+        super(ctx, frameSize);
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        if (buffer.position() > 0 && buffer.hasRemaining()) {
+            movePartialFutureToStartPosition();
+        } else {
+            buffer.clear();
+        }
+    }
+
+    private void movePartialFutureToStartPosition() {
+        assert buffer.hasArray();
+        if (!FrameHelper.hasBeenCleared(buffer, buffer.position())) {
+            buffer.compact();
+            FrameHelper.clearRemainingFrame(buffer, buffer.position()); // mark it to make reset idempotent
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
new file mode 100644
index 0000000..01744f9
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.util;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Basically it a union of the {@link IFrameTupleAccessor} and {@link IFrameTupleAppender}.
+ * Moreover, it has the delete function as well.
+ * This is a special TupleAccessor used for TopK sorting.
+ * In HeapSort, or other Tuple-based operators, we need to append the tuple, access the arbitrary previously
+ * inserted tuple, and delete the previously inserted tuple.
+ */
+public interface IAppendDeletableFrameTupleAccessor extends IFrameTupleAccessor {
+
+    /**
+     * Prepare to write on this buffer
+     *
+     * @param buffer
+     * @throws HyracksDataException
+     */
+    void clear(ByteBuffer buffer) throws HyracksDataException;
+
+    /**
+     * Append tuple content to this buffer. Return the new tid as a handle to the caller.
+     *
+     * @param tupleAccessor
+     * @param tIndex
+     * @return
+     * @throws HyracksDataException
+     */
+    int append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException;
+
+    /**
+     * Remove the certain tuple by tid
+     *
+     * @param tid
+     */
+    void delete(int tid);
+
+    /**
+     * Reorganize the space to remove the unused space and make the free space contiguous.
+     */
+    void reOrganizeBuffer();
+
+    /**
+     * @return how many free space in total in the buffer, including the fragmented space
+     */
+    int getTotalFreeSpace();
+
+    /**
+     * @return how many contiguous free space in the buffer.
+     */
+    int getContiguousFreeSpace();
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstractHeap.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstractHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstractHeap.java
new file mode 100644
index 0000000..5d8b252
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstractHeap.java
@@ -0,0 +1,156 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import java.util.Arrays;
+
+import edu.uci.ics.hyracks.dataflow.std.util.MathUtil;
+
+public abstract class AbstractHeap implements IHeap<IResetableComparable> {
+    protected static final int NOT_EXIST = -1;
+    protected static final int MAX_INITIAL_CAPACITY = 1024;
+    protected IResetableComparable[] entries;
+    protected IResetableComparable tempEntry;
+    protected IResetableComparableFactory factory;
+    protected int numEntry;
+
+    public AbstractHeap(IResetableComparableFactory factory, int capacity) {
+        capacity = Math.min(MAX_INITIAL_CAPACITY, Math.max(1, capacity));
+        this.entries = new IResetableComparable[capacity];
+        this.numEntry = 0;
+        this.tempEntry = factory.createResetableComparable();
+        this.factory = factory;
+    }
+
+    @Override
+    public void insert(IResetableComparable element) {
+        if (numEntry >= entries.length) {
+            entries = Arrays.copyOf(entries, entries.length * 2);
+        }
+        if (entries[numEntry] == null) {
+            entries[numEntry] = factory.createResetableComparable();
+        }
+        entries[numEntry++].reset(element);
+        bubbleUp(numEntry - 1);
+    }
+
+    protected abstract void bubbleUp(int i);
+
+    protected abstract void trickleDown(int i);
+
+    protected void swap(int cid, int pid) {
+        tempEntry.reset(entries[cid]);
+        entries[cid].reset(entries[pid]);
+        entries[pid].reset(tempEntry);
+    }
+
+    protected int compareTo(int i, int j) {
+        return entries[i].compareTo(entries[j]);
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return numEntry == 0;
+    }
+
+    @Override
+    public void reset() {
+        for (int i = 0; i < numEntry; i++) {
+            entries[i] = null;
+        }
+        numEntry = 0;
+    }
+
+    /**
+     * By getting the entries it can manipulate the entries which may violate the Heap property.
+     * Use with care.
+     *
+     * @return
+     */
+    @Deprecated
+    public IResetableComparable[] getEntries() {
+        return entries;
+    }
+
+    @Override
+    public int getNumEntries() {
+        return numEntry;
+    }
+
+    protected int getLevel(int cid) {
+        return MathUtil.log2Floor(cid + 1);
+    }
+
+    static int getParentId(int cid) {
+        return cid < 1 ? NOT_EXIST : (cid - 1) / 2;
+    }
+
+    static int getLeftChild(int id, int numEntry) {
+        int cid = id * 2 + 1;
+        return cid >= numEntry ? NOT_EXIST : cid;
+    }
+
+    protected int getLeftChild(int id) {
+        return getLeftChild(id, numEntry);
+    }
+
+    static int getRightChild(int id, int numEntry) {
+        int cid = id * 2 + 2;
+        return cid >= numEntry ? NOT_EXIST : cid;
+    }
+
+    protected int getRightChild(int id) {
+        return getRightChild(id, numEntry);
+    }
+
+    protected int getGrandParentId(int id) {
+        int pid = getParentId(id);
+        return pid == NOT_EXIST ? NOT_EXIST : getParentId(pid);
+    }
+
+    protected boolean isDirectChild(int id, int childId) {
+        return id == getParentId(childId);
+    }
+
+    protected int getMinChild(int id) {
+        int min = NOT_EXIST;
+        if (id != NOT_EXIST) {
+            min = getLeftChild(id, numEntry);
+            if (min != NOT_EXIST) {
+                int rightCid = getRightChild(id, numEntry);
+                if (rightCid != NOT_EXIST) {
+                    min = compareTo(rightCid, min) < 0 ? rightCid : min;
+                }
+            }
+        }
+        return min;
+    }
+
+    protected int getMaxChild(int id) {
+        int max = NOT_EXIST;
+        if (id != NOT_EXIST) {
+            max = getLeftChild(id, numEntry);
+            if (max != NOT_EXIST) {
+                int rightCid = getRightChild(id, numEntry);
+                if (rightCid != NOT_EXIST) {
+                    max = compareTo(rightCid, max) > 0 ? rightCid : max;
+                }
+            }
+        }
+        return max;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IHeap.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IHeap.java
new file mode 100644
index 0000000..f650f12
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IHeap.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IHeap<T> {
+    /**
+     * Inserts a new element into the selectionTree
+     *
+     * @param element to be inserted
+     */
+    void insert(T element);
+
+    /**
+     * @return True of the selection tree does not have any element, false
+     * otherwise
+     */
+    boolean isEmpty();
+
+    /**
+     * Removes all the elements in the tree
+     */
+    void reset();
+
+    /**
+     * Return the number of the inserted tuples
+     *
+     * @return
+     */
+    int getNumEntries();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMaxHeap.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMaxHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMaxHeap.java
new file mode 100644
index 0000000..d932a0e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMaxHeap.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IMaxHeap<T> extends IHeap<T> {
+    /**
+     * Removes and returns the largest element in the tree.
+     * Make sure the heap is not empty (by {@link #isEmpty()}) before calling this method
+     *
+     * @param result
+     */
+    void getMax(T result);
+
+    /**
+     * Returns (and does NOT remove) the largest element in the tree
+     *
+     * @param result is the object that will eventually contain maximum entry
+     *               pointer
+     */
+    void peekMax(T result);
+
+    /**
+     * Removes the current max and insert a new element.
+     * Normally it is a faster way to call getMax() && insert() together
+     *
+     * @param newElement
+     */
+    void replaceMax(T newElement);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinHeap.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinHeap.java
new file mode 100644
index 0000000..784c492
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinHeap.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IMinHeap<T> extends IHeap<T> {
+    /**
+     * Removes and returns the smallest element in the tree.
+     * Make sure the heap is not empty (by {@link #isEmpty()}) before calling this method
+     *
+     * @param result
+     */
+    void getMin(T result);
+
+    /**
+     * Returns (and does NOT remove) the smallest element in the tree
+     *
+     * @param result is the object that will eventually contain minimum entry
+     *               pointer
+     */
+    void peekMin(T result);
+
+    /**
+     * Removes the current min and insert a new element.
+     * Normally it is a faster way to call getMin() && insert() together
+     *
+     * @param newElement
+     */
+    void replaceMin(T newElement);
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinMaxHeap.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinMaxHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinMaxHeap.java
new file mode 100644
index 0000000..8225d88
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinMaxHeap.java
@@ -0,0 +1,18 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IMinMaxHeap<T> extends IMinHeap<T>, IMaxHeap<T> {
+}
\ No newline at end of file


[02/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java b/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
index e46e685..790552c 100644
--- a/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
+++ b/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
@@ -19,10 +19,10 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.Reader;
-import java.nio.ByteBuffer;
 import java.util.Arrays;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -42,9 +42,7 @@ public class WordTupleParserFactory implements ITupleParserFactory {
             @Override
             public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
                 try {
-                    ByteBuffer frame = ctx.allocateFrame();
-                    FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-                    appender.reset(frame, true);
+                    FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
                     ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
                     DataOutput dos = tb.getDataOutput();
 
@@ -54,17 +52,10 @@ public class WordTupleParserFactory implements ITupleParserFactory {
                         tb.reset();
                         utf8StringParser.parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart, dos);
                         tb.addFieldEndOffset();
-                        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                            FrameUtils.flushFrame(frame, writer);
-                            appender.reset(frame, true);
-                            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                                throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
-                            }
-                        }
-                    }
-                    if (appender.getTupleCount() > 0) {
-                        FrameUtils.flushFrame(frame, writer);
+                        FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+                                tb.getSize());
                     }
+                    appender.flush(writer, true);
                 } catch (IOException e) {
                     throw new HyracksDataException(e);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
index a437f37..e22f27f 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -56,7 +56,7 @@
             <configuration>
               <programs>
                 <program>
-                  <mainClass>edu.uci.ics.hyracks.examples.tpch.client.Main</mainClass>
+                  <mainClass>edu.uci.ics.hyracks.examples.tpch.client.Sort</mainClass>
                   <name>tpchclient</name>
                 </program>
               </programs>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Common.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Common.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Common.java
new file mode 100644
index 0000000..17f1d3b
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Common.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.examples.tpch.client;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+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.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+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.std.file.FileSplit;
+
+public class Common {
+    static RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+    static RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE });
+    static RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE });
+
+    static IValueParserFactory[] orderParserFactories = new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE };
+
+
+    static FileSplit[] parseFileSplits(String fileSplits) {
+        String[] splits = fileSplits.split(",");
+        FileSplit[] fSplits = new FileSplit[splits.length];
+        for (int i = 0; i < splits.length; ++i) {
+            String s = splits[i].trim();
+            int idx = s.indexOf(':');
+            if (idx < 0) {
+                throw new IllegalArgumentException("File split " + s + " not well formed");
+            }
+            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
+        }
+        return fSplits;
+    }
+
+    static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
+        String[] parts = new String[splits.length];
+        for (int i = 0; i < splits.length; ++i) {
+            parts[i] = splits[i].getNodeName();
+        }
+        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Join.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Join.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Join.java
new file mode 100644
index 0000000..748d809
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Join.java
@@ -0,0 +1,320 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.examples.tpch.client;
+
+import static edu.uci.ics.hyracks.examples.tpch.client.Common.*;
+
+import java.util.EnumSet;
+
+import org.kohsuke.args4j.CmdLineParser;
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
+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.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+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.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.FrameFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+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.group.hash.HashGroupOperatorDescriptor;
+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;
+import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
+
+public class Join {
+    private static class Options {
+        @Option(name = "-host", usage = "Hyracks Cluster Controller Host name", required = true)
+        public String host;
+
+        @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)", required = false)
+        public int port = 1098;
+
+        @Option(name = "-infile-customer-splits", usage = "Comma separated list of file-splits for the CUSTOMER input. A file-split is <node-name>:<path>", required = true)
+        public String inFileCustomerSplits;
+
+        @Option(name = "-infile-order-splits", usage = "Comma separated list of file-splits for the ORDER input. A file-split is <node-name>:<path>", required = true)
+        public String inFileOrderSplits;
+
+        @Option(name = "-outfile-splits", usage = "Comma separated list of file-splits for the output", required = true)
+        public String outFileSplits;
+
+        @Option(name = "-num-join-partitions", usage = "Number of Join partitions to use (default: 1)", required = false)
+        public int numJoinPartitions = 1;
+
+        @Option(name = "-profile", usage = "Enable/Disable profiling. (default: enabled)")
+        public boolean profile = true;
+
+        @Option(name = "-table-size", usage = "Table size for in-memory hash join", required = false)
+        public int tableSize = 8191;
+
+        @Option(name = "-algo", usage = "Join types", required = true)
+        public String algo;
+
+        // For grace/hybrid hash join only
+        @Option(name = "-mem-size", usage = "Memory size for hash join", required = true)
+        public int memSize;
+
+        @Option(name = "-input-size", usage = "Input size of the grace/hybrid hash join", required = false)
+        public int graceInputSize = 10;
+
+        @Option(name = "-records-per-frame", usage = "Records per frame for grace/hybrid hash join", required = false)
+        public int graceRecordsPerFrame = 200;
+
+        @Option(name = "-grace-factor", usage = "Factor of the grace/hybrid hash join", required = false)
+        public double graceFactor = 1.2;
+
+        // Whether group-by is processed after the join
+        @Option(name = "-has-groupby", usage = "Whether to have group-by operation after join (default: disabled)", required = false)
+        public boolean hasGroupBy = false;
+
+        @Option(name = "-frame-size", usage = "Hyracks frame size (default: 32768)", required = false)
+        public int frameSize = 32768;
+    }
+
+    public static void main(String[] args) throws Exception {
+        Options options = new Options();
+        CmdLineParser parser = new CmdLineParser(options);
+        parser.parseArgument(args);
+
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
+
+        JobSpecification job = createJob(parseFileSplits(options.inFileCustomerSplits),
+                parseFileSplits(options.inFileOrderSplits), parseFileSplits(options.outFileSplits),
+                options.numJoinPartitions, options.algo, options.graceInputSize, options.graceRecordsPerFrame,
+                options.graceFactor, options.memSize, options.tableSize, options.hasGroupBy, options.frameSize);
+
+        long start = System.currentTimeMillis();
+        JobId jobId = hcc.startJob(job,
+                options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
+        hcc.waitForCompletion(jobId);
+        long end = System.currentTimeMillis();
+        System.err.println(start + " " + end + " " + (end - start));
+    }
+
+    private static JobSpecification createJob(FileSplit[] customerSplits, FileSplit[] orderSplits,
+            FileSplit[] resultSplits, int numJoinPartitions, String algo, int graceInputSize, int graceRecordsPerFrame,
+            double graceFactor, int memSize, int tableSize, boolean hasGroupBy, int frameSize)
+            throws HyracksDataException {
+        JobSpecification spec = new JobSpecification(frameSize);
+
+        IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(customerSplits);
+
+        IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(orderSplits);
+
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), Common.ordersDesc);
+        createPartitionConstraint(spec, ordScanner, orderSplits);
+
+        FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
+                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+                        UTF8StringParserFactory.INSTANCE }, '|'), Common.custDesc);
+        createPartitionConstraint(spec, custScanner, customerSplits);
+
+        IOperatorDescriptor join;
+
+        if ("nestedloop".equalsIgnoreCase(algo)) {
+            join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
+                    PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), Common.custOrderJoinDesc,
+                    memSize, false, null);
+
+        } else if ("gracehash".equalsIgnoreCase(algo)) {
+            join = new GraceHashJoinOperatorDescriptor(
+                    spec,
+                    memSize,
+                    graceInputSize,
+                    graceRecordsPerFrame,
+                    graceFactor,
+                    new int[] { 0 },
+                    new int[] { 1 },
+                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                            .of(UTF8StringPointable.FACTORY) },
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                    Common.custOrderJoinDesc, null);
+
+        } else if ("hybridhash".equalsIgnoreCase(algo)) {
+            join = new HybridHashJoinOperatorDescriptor(
+                    spec,
+                    memSize,
+                    graceInputSize,
+                    graceRecordsPerFrame,
+                    graceFactor,
+                    new int[] { 0 },
+                    new int[] { 1 },
+                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                            .of(UTF8StringPointable.FACTORY) },
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                    Common.custOrderJoinDesc, null);
+
+        } else {
+            join = new InMemoryHashJoinOperatorDescriptor(
+                    spec,
+                    new int[] { 0 },
+                    new int[] { 1 },
+                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                            .of(UTF8StringPointable.FACTORY) },
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                    Common.custOrderJoinDesc, 6000000, null);
+        }
+
+        PartitionConstraintHelper.addPartitionCountConstraint(spec, join, numJoinPartitions);
+
+        IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 1 },
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
+        spec.connect(ordJoinConn, ordScanner, 0, join, 1);
+
+        IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
+                new FieldHashPartitionComputerFactory(new int[] { 0 },
+                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                .of(UTF8StringPointable.FACTORY) }));
+        spec.connect(custJoinConn, custScanner, 0, join, 0);
+
+        IOperatorDescriptor endingOp = join;
+
+        if (hasGroupBy) {
+
+            RecordDescriptor groupResultDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+                    UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
+
+            HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(
+                    spec,
+                    new int[] { 6 },
+                    new FieldHashPartitionComputerFactory(new int[] { 6 },
+                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                    .of(UTF8StringPointable.FACTORY) }),
+                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+                    new MultiFieldsAggregatorFactory(
+                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
+                    groupResultDesc, 16);
+            createPartitionConstraint(spec, gby, resultSplits);
+
+            IConnectorDescriptor joinGroupConn = new MToNPartitioningConnectorDescriptor(spec,
+                    new FieldHashPartitionComputerFactory(new int[] { 6 },
+                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+                                    .of(UTF8StringPointable.FACTORY) }));
+            spec.connect(joinGroupConn, join, 0, gby, 0);
+
+            endingOp = gby;
+        }
+
+        IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(resultSplits);
+        FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
+        createPartitionConstraint(spec, writer, resultSplits);
+
+        IConnectorDescriptor endingPrinterConn = new OneToOneConnectorDescriptor(spec);
+        spec.connect(endingPrinterConn, endingOp, 0, writer, 0);
+
+        spec.addRoot(writer);
+        return spec;
+    }
+
+
+
+    static class JoinComparatorFactory implements ITuplePairComparatorFactory {
+        private static final long serialVersionUID = 1L;
+
+        private final IBinaryComparatorFactory bFactory;
+        private final int pos0;
+        private final int pos1;
+
+        public JoinComparatorFactory(IBinaryComparatorFactory bFactory, int pos0, int pos1) {
+            this.bFactory = bFactory;
+            this.pos0 = pos0;
+            this.pos1 = pos1;
+        }
+
+        @Override
+        public ITuplePairComparator createTuplePairComparator(IHyracksTaskContext ctx) {
+            return new JoinComparator(bFactory.createBinaryComparator(), pos0, pos1);
+        }
+    }
+
+    static class JoinComparator implements ITuplePairComparator {
+
+        private final IBinaryComparator bComparator;
+        private final int field0;
+        private final int field1;
+
+        public JoinComparator(IBinaryComparator bComparator, int field0, int field1) {
+            this.bComparator = bComparator;
+            this.field0 = field0;
+            this.field1 = field1;
+        }
+
+        @Override
+        public int compare(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
+                throws HyracksDataException {
+            int tStart0 = accessor0.getTupleStartOffset(tIndex0);
+            int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
+
+            int tStart1 = accessor1.getTupleStartOffset(tIndex1);
+            int fStartOffset1 = accessor1.getFieldSlotsLength() + tStart1;
+
+            int fStart0 = accessor0.getFieldStartOffset(tIndex0, field0);
+            int fEnd0 = accessor0.getFieldEndOffset(tIndex0, field0);
+            int fLen0 = fEnd0 - fStart0;
+
+            int fStart1 = accessor1.getFieldStartOffset(tIndex1, field1);
+            int fEnd1 = accessor1.getFieldEndOffset(tIndex1, field1);
+            int fLen1 = fEnd1 - fStart1;
+
+            int c = bComparator.compare(accessor0.getBuffer().array(), fStart0 + fStartOffset0, fLen0, accessor1
+                    .getBuffer().array(), fStart1 + fStartOffset1, fLen1);
+            if (c != 0) {
+                return c;
+            }
+            return 0;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
deleted file mode 100644
index 1d4e6ce..0000000
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ /dev/null
@@ -1,362 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.examples.tpch.client;
-
-import java.io.File;
-import java.util.EnumSet;
-
-import org.kohsuke.args4j.CmdLineParser;
-import org.kohsuke.args4j.Option;
-
-import edu.uci.ics.hyracks.api.client.HyracksConnection;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobId;
-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.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-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.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.FrameFileWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-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.group.hash.HashGroupOperatorDescriptor;
-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;
-import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
-
-public class Main {
-    private static class Options {
-        @Option(name = "-host", usage = "Hyracks Cluster Controller Host name", required = true)
-        public String host;
-
-        @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)", required = false)
-        public int port = 1098;
-
-        @Option(name = "-infile-customer-splits", usage = "Comma separated list of file-splits for the CUSTOMER input. A file-split is <node-name>:<path>", required = true)
-        public String inFileCustomerSplits;
-
-        @Option(name = "-infile-order-splits", usage = "Comma separated list of file-splits for the ORDER input. A file-split is <node-name>:<path>", required = true)
-        public String inFileOrderSplits;
-
-        @Option(name = "-outfile-splits", usage = "Comma separated list of file-splits for the output", required = true)
-        public String outFileSplits;
-
-        @Option(name = "-num-join-partitions", usage = "Number of Join partitions to use (default: 1)", required = false)
-        public int numJoinPartitions = 1;
-
-        @Option(name = "-profile", usage = "Enable/Disable profiling. (default: enabled)")
-        public boolean profile = true;
-
-        @Option(name = "-table-size", usage = "Table size for in-memory hash join", required = false)
-        public int tableSize = 8191;
-
-        @Option(name = "-algo", usage = "Join types", required = true)
-        public String algo;
-
-        // For grace/hybrid hash join only
-        @Option(name = "-mem-size", usage = "Memory size for hash join", required = true)
-        public int memSize;
-
-        @Option(name = "-input-size", usage = "Input size of the grace/hybrid hash join", required = false)
-        public int graceInputSize = 10;
-
-        @Option(name = "-records-per-frame", usage = "Records per frame for grace/hybrid hash join", required = false)
-        public int graceRecordsPerFrame = 200;
-
-        @Option(name = "-grace-factor", usage = "Factor of the grace/hybrid hash join", required = false)
-        public double graceFactor = 1.2;
-
-        // Whether group-by is processed after the join
-        @Option(name = "-has-groupby", usage = "Whether to have group-by operation after join (default: disabled)", required = false)
-        public boolean hasGroupBy = false;
-
-        @Option(name = "-frame-size", usage = "Hyracks frame size (default: 32768)", required = false)
-        public int frameSize = 32768;
-    }
-
-    public static void main(String[] args) throws Exception {
-        Options options = new Options();
-        CmdLineParser parser = new CmdLineParser(options);
-        parser.parseArgument(args);
-
-        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
-
-        JobSpecification job = createJob(parseFileSplits(options.inFileCustomerSplits),
-                parseFileSplits(options.inFileOrderSplits), parseFileSplits(options.outFileSplits),
-                options.numJoinPartitions, options.algo, options.graceInputSize, options.graceRecordsPerFrame,
-                options.graceFactor, options.memSize, options.tableSize, options.hasGroupBy, options.frameSize);
-
-        long start = System.currentTimeMillis();
-        JobId jobId = hcc.startJob(job,
-                options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
-        hcc.waitForCompletion(jobId);
-        long end = System.currentTimeMillis();
-        System.err.println(start + " " + end + " " + (end - start));
-    }
-
-    private static FileSplit[] parseFileSplits(String fileSplits) {
-        String[] splits = fileSplits.split(",");
-        FileSplit[] fSplits = new FileSplit[splits.length];
-        for (int i = 0; i < splits.length; ++i) {
-            String s = splits[i].trim();
-            int idx = s.indexOf(':');
-            if (idx < 0) {
-                throw new IllegalArgumentException("File split " + s + " not well formed");
-            }
-            fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
-        }
-        return fSplits;
-    }
-
-    private static JobSpecification createJob(FileSplit[] customerSplits, FileSplit[] orderSplits,
-            FileSplit[] resultSplits, int numJoinPartitions, String algo, int graceInputSize, int graceRecordsPerFrame,
-            double graceFactor, int memSize, int tableSize, boolean hasGroupBy, int frameSize)
-            throws HyracksDataException {
-        JobSpecification spec = new JobSpecification(frameSize);
-
-        IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(customerSplits);
-        RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
-
-        IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(orderSplits);
-        RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
-                UTF8StringSerializerDeserializer.INSTANCE });
-
-        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
-        createPartitionConstraint(spec, ordScanner, orderSplits);
-
-        FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
-                new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
-                        UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
-        createPartitionConstraint(spec, custScanner, customerSplits);
-
-        IOperatorDescriptor join;
-
-        if ("nestedloop".equalsIgnoreCase(algo)) {
-            join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
-                    PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), custOrderJoinDesc,
-                    memSize, false, null);
-
-        } else if ("gracehash".equalsIgnoreCase(algo)) {
-            join = new GraceHashJoinOperatorDescriptor(
-                    spec,
-                    memSize,
-                    graceInputSize,
-                    graceRecordsPerFrame,
-                    graceFactor,
-                    new int[] { 0 },
-                    new int[] { 1 },
-                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                            .of(UTF8StringPointable.FACTORY) },
-                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    custOrderJoinDesc, null);
-
-        } else if ("hybridhash".equalsIgnoreCase(algo)) {
-            join = new HybridHashJoinOperatorDescriptor(
-                    spec,
-                    memSize,
-                    graceInputSize,
-                    graceRecordsPerFrame,
-                    graceFactor,
-                    new int[] { 0 },
-                    new int[] { 1 },
-                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                            .of(UTF8StringPointable.FACTORY) },
-                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    custOrderJoinDesc, null);
-
-        } else {
-            join = new InMemoryHashJoinOperatorDescriptor(
-                    spec,
-                    new int[] { 0 },
-                    new int[] { 1 },
-                    new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                            .of(UTF8StringPointable.FACTORY) },
-                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    custOrderJoinDesc, 6000000, null);
-        }
-
-        PartitionConstraintHelper.addPartitionCountConstraint(spec, join, numJoinPartitions);
-
-        IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(new int[] { 1 },
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
-        spec.connect(ordJoinConn, ordScanner, 0, join, 1);
-
-        IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
-                new FieldHashPartitionComputerFactory(new int[] { 0 },
-                        new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                .of(UTF8StringPointable.FACTORY) }));
-        spec.connect(custJoinConn, custScanner, 0, join, 0);
-
-        IOperatorDescriptor endingOp = join;
-
-        if (hasGroupBy) {
-
-            RecordDescriptor groupResultDesc = new RecordDescriptor(new ISerializerDeserializer[] {
-                    UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
-
-            HashGroupOperatorDescriptor gby = new HashGroupOperatorDescriptor(
-                    spec,
-                    new int[] { 6 },
-                    new FieldHashPartitionComputerFactory(new int[] { 6 },
-                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                    .of(UTF8StringPointable.FACTORY) }),
-                    new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
-                    new MultiFieldsAggregatorFactory(
-                            new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
-                    groupResultDesc, 16);
-            createPartitionConstraint(spec, gby, resultSplits);
-
-            IConnectorDescriptor joinGroupConn = new MToNPartitioningConnectorDescriptor(spec,
-                    new FieldHashPartitionComputerFactory(new int[] { 6 },
-                            new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
-                                    .of(UTF8StringPointable.FACTORY) }));
-            spec.connect(joinGroupConn, join, 0, gby, 0);
-
-            endingOp = gby;
-        }
-
-        IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(resultSplits);
-        FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
-        createPartitionConstraint(spec, writer, resultSplits);
-
-        IConnectorDescriptor endingPrinterConn = new OneToOneConnectorDescriptor(spec);
-        spec.connect(endingPrinterConn, endingOp, 0, writer, 0);
-
-        spec.addRoot(writer);
-        return spec;
-    }
-
-    private static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
-        String[] parts = new String[splits.length];
-        for (int i = 0; i < splits.length; ++i) {
-            parts[i] = splits[i].getNodeName();
-        }
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
-    }
-
-    static class JoinComparatorFactory implements ITuplePairComparatorFactory {
-        private static final long serialVersionUID = 1L;
-
-        private final IBinaryComparatorFactory bFactory;
-        private final int pos0;
-        private final int pos1;
-
-        public JoinComparatorFactory(IBinaryComparatorFactory bFactory, int pos0, int pos1) {
-            this.bFactory = bFactory;
-            this.pos0 = pos0;
-            this.pos1 = pos1;
-        }
-
-        @Override
-        public ITuplePairComparator createTuplePairComparator(IHyracksTaskContext ctx) {
-            return new JoinComparator(bFactory.createBinaryComparator(), pos0, pos1);
-        }
-    }
-
-    static class JoinComparator implements ITuplePairComparator {
-
-        private final IBinaryComparator bComparator;
-        private final int field0;
-        private final int field1;
-
-        public JoinComparator(IBinaryComparator bComparator, int field0, int field1) {
-            this.bComparator = bComparator;
-            this.field0 = field0;
-            this.field1 = field1;
-        }
-
-        @Override
-        public int compare(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
-                throws HyracksDataException {
-            int tStart0 = accessor0.getTupleStartOffset(tIndex0);
-            int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
-
-            int tStart1 = accessor1.getTupleStartOffset(tIndex1);
-            int fStartOffset1 = accessor1.getFieldSlotsLength() + tStart1;
-
-            int fStart0 = accessor0.getFieldStartOffset(tIndex0, field0);
-            int fEnd0 = accessor0.getFieldEndOffset(tIndex0, field0);
-            int fLen0 = fEnd0 - fStart0;
-
-            int fStart1 = accessor1.getFieldStartOffset(tIndex1, field1);
-            int fEnd1 = accessor1.getFieldEndOffset(tIndex1, field1);
-            int fLen1 = fEnd1 - fStart1;
-
-            int c = bComparator.compare(accessor0.getBuffer().array(), fStart0 + fStartOffset0, fLen0, accessor1
-                    .getBuffer().array(), fStart1 + fStartOffset1, fLen1);
-            if (c != 0) {
-                return c;
-            }
-            return 0;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Sort.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Sort.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Sort.java
new file mode 100644
index 0000000..7570b0b
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Sort.java
@@ -0,0 +1,165 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.examples.tpch.client;
+
+import static edu.uci.ics.hyracks.examples.tpch.client.Common.createPartitionConstraint;
+import static edu.uci.ics.hyracks.examples.tpch.client.Common.orderParserFactories;
+import static edu.uci.ics.hyracks.examples.tpch.client.Common.ordersDesc;
+import static edu.uci.ics.hyracks.examples.tpch.client.Common.parseFileSplits;
+
+import java.util.EnumSet;
+
+import org.kohsuke.args4j.CmdLineParser;
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
+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.normalizers.UTF8StringNormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.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.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSorterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.TopKSorterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
+
+public class Sort {
+    private static class Options {
+        @Option(name = "-host", usage = "Hyracks Cluster Controller Host name", required = true)
+        public String host;
+
+        @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)", required = false)
+        public int port = 1098;
+
+        @Option(name = "-frame-size", usage = "Hyracks frame size (default: 32768)", required = false)
+        public int frameSize = 32768;
+
+        @Option(name = "-frame-limit", usage = "memory limit for sorting (default: 4)", required = false)
+        public int frameLimit = 4;
+
+        @Option(name = "-infile-splits", usage = "Comma separated list of file-splits for the ORDER input. A file-split is <node-name>:<path>", required = true)
+        public String inFileOrderSplits;
+
+        @Option(name = "-outfile-splits", usage = "Comma separated list of file-splits for the output", required = true)
+        public String outFileSplits;
+
+        @Option(name = "-membuffer-alg", usage = "bestfit or lastfit (default: lastfit)", required = false)
+        public String memBufferAlg = "lastfit";
+
+        @Option(name = "-profile", usage = "Enable/Disable profiling. (default: enabled)")
+        public boolean profile = true;
+
+        @Option(name = "-topK", usage = "only output topK for each node. (default: not set)")
+        public int topK = Integer.MAX_VALUE;
+
+        @Option(name = "-heapSort", usage = "using heap sort for topK result. (default: false)")
+        public boolean usingHeapSorter = false;
+    }
+
+    static int[] SortFields = new int[] { 1, 0 };
+    static IBinaryComparatorFactory[] SortFieldsComparatorFactories = new IBinaryComparatorFactory[] {
+            PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+            PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
+
+    static IBinaryHashFunctionFactory[] orderBinaryHashFunctionFactories = new IBinaryHashFunctionFactory[] {
+            PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+            PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) };
+
+    public static void main(String[] args) throws Exception {
+        Options options = new Options();
+        CmdLineParser parser = new CmdLineParser(options);
+        if (args.length == 0) {
+            parser.printUsage(System.err);
+            return;
+        }
+        parser.parseArgument(args);
+
+        IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
+
+        JobSpecification job = createJob(parseFileSplits(options.inFileOrderSplits),
+                parseFileSplits(options.outFileSplits),
+                options.memBufferAlg, options.frameLimit, options.frameSize, options.topK, options.usingHeapSorter);
+
+        long start = System.currentTimeMillis();
+        JobId jobId = hcc.startJob(job,
+                options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
+        hcc.waitForCompletion(jobId);
+        long end = System.currentTimeMillis();
+        System.err.println("finished in:" + (end - start) + "ms");
+    }
+
+    private static JobSpecification createJob(FileSplit[] ordersSplits, FileSplit[] outputSplit, String memBufferAlg,
+            int frameLimit, int frameSize, int limit, boolean usingHeapSorter) {
+        JobSpecification spec = new JobSpecification();
+
+        spec.setFrameSize(frameSize);
+        IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+        FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+                new DelimitedDataTupleParserFactory(orderParserFactories, '|'), ordersDesc);
+        createPartitionConstraint(spec, ordScanner, ordersSplits);
+        AbstractSorterOperatorDescriptor sorter;
+        if (usingHeapSorter && limit < Integer.MAX_VALUE) {
+            sorter = new TopKSorterOperatorDescriptor(spec, frameLimit, limit, SortFields, null,
+                    SortFieldsComparatorFactories, ordersDesc);
+        } else {
+            if (memBufferAlg.equalsIgnoreCase("bestfit")) {
+                sorter = new ExternalSortOperatorDescriptor(spec, frameLimit, SortFields,
+                        null, SortFieldsComparatorFactories, ordersDesc, Algorithm.MERGE_SORT,
+                        EnumFreeSlotPolicy.SMALLEST_FIT, limit);
+            } else if (memBufferAlg.equalsIgnoreCase("biggestfit")) {
+                sorter = new ExternalSortOperatorDescriptor(spec, frameLimit, SortFields, null,
+                        SortFieldsComparatorFactories, ordersDesc, Algorithm.MERGE_SORT, EnumFreeSlotPolicy.BIGGEST_FIT,
+                        limit);
+            } else {
+                sorter = new ExternalSortOperatorDescriptor(spec, frameLimit, SortFields, null,
+                        SortFieldsComparatorFactories, ordersDesc, Algorithm.MERGE_SORT, EnumFreeSlotPolicy.LAST_FIT,
+                        limit);
+
+            }
+        }
+        createPartitionConstraint(spec, sorter, ordersSplits);
+        IFileSplitProvider outputSplitProvider = new ConstantFileSplitProvider(outputSplit);
+        IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outputSplitProvider, "|");
+        createPartitionConstraint(spec, printer, outputSplit);
+
+        spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
+
+        spec.connect(
+                new MToNPartitioningMergingConnectorDescriptor(spec,
+                        new FieldHashPartitionComputerFactory(SortFields, orderBinaryHashFunctionFactories),
+                        SortFields, SortFieldsComparatorFactories, new UTF8StringNormalizedKeyComputerFactory()),
+                sorter, 0, printer, 0);
+
+        spec.addRoot(printer);
+        return spec;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
index 4e48e9b..cb1ca87 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
@@ -76,7 +76,7 @@ public class HDFSWriteOperatorDescriptor extends AbstractSingleActivityOperatorD
 
             private FSDataOutputStream dos;
             private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
-            private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+            private FrameTupleAccessor accessor = new FrameTupleAccessor(inputRd);
             private FrameTupleReference tuple = new FrameTupleReference();
             private ITupleWriter tupleWriter;
             private ClassLoader ctxCL;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
index 92cde9d..62cd76a 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
@@ -15,12 +15,11 @@
 
 package edu.uci.ics.hyracks.hdfs.lib;
 
-import java.nio.ByteBuffer;
-
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -37,9 +36,7 @@ public class TextKeyValueParserFactory implements IKeyValueParserFactory<LongWri
             throws HyracksDataException {
 
         final ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
-        final ByteBuffer buffer = ctx.allocateFrame();
-        final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-        appender.reset(buffer, true);
+        final FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
 
         return new IKeyValueParser<LongWritable, Text>() {
 
@@ -53,18 +50,13 @@ public class TextKeyValueParserFactory implements IKeyValueParserFactory<LongWri
                     throws HyracksDataException {
                 tb.reset();
                 tb.addField(value.getBytes(), 0, value.getLength());
-                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    FrameUtils.flushFrame(buffer, writer);
-                    appender.reset(buffer, true);
-                    if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                        throw new HyracksDataException("tuple cannot be appended into the frame");
-                    }
-                }
+                FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+                        tb.getSize());
             }
 
             @Override
             public void close(IFrameWriter writer) throws HyracksDataException {
-                FrameUtils.flushFrame(buffer, writer);
+                appender.flush(writer, false);
             }
 
         };

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
index 068cdfc..0d24ee5 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
@@ -75,7 +75,7 @@ public class HDFSWriteOperatorDescriptor extends AbstractSingleActivityOperatorD
 
             private FSDataOutputStream dos;
             private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
-            private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+            private FrameTupleAccessor accessor = new FrameTupleAccessor(inputRd);
             private FrameTupleReference tuple = new FrameTupleReference();
             private ITupleWriter tupleWriter;
             private ClassLoader ctxCL;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
index e60026e..785258f 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -17,7 +17,6 @@ package edu.uci.ics.hyracks.storage.am.btree.dataflow;
 
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -64,14 +63,8 @@ public class BTreeUpdateSearchOperatorNodePushable extends BTreeSearchOperatorNo
                 dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
                 tb.addFieldEndOffset();
             }
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                FrameUtils.flushFrame(writeBuffer, writer);
-                appender.reset(writeBuffer, true);
-                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
-                            + appender.getBuffer().capacity() + ")");
-                }
-            }
+            FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+                    tb.getSize());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
index 9f3c5dd..9aec4cb 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
@@ -66,7 +66,7 @@ public class IndexBulkLoadOperatorNodePushable extends
     public void open() throws HyracksDataException {
         RecordDescriptor recDesc = recDescProvider.getInputRecordDescriptor(
                 opDesc.getActivityId(), 0);
-        accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+        accessor = new FrameTupleAccessor(recDesc);
         indexHelper.open();
         index = indexHelper.getIndexInstance();
         try {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
index 092fada..48a395b 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
@@ -16,6 +16,8 @@ package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -45,7 +47,7 @@ public class IndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryIn
     protected final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
     protected FrameTupleAccessor accessor;
     protected FrameTupleReference frameTuple;
-    protected ByteBuffer writeBuffer;
+    protected IFrame writeBuffer;
     protected IIndexAccessor indexAccessor;
     protected ITupleFilter tupleFilter;
     protected IModificationOperationCallback modCallback;
@@ -63,8 +65,8 @@ public class IndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryIn
     @Override
     public void open() throws HyracksDataException {
         RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
-        writeBuffer = ctx.allocateFrame();
+        accessor = new FrameTupleAccessor(inputRecDesc);
+        writeBuffer = new VSizeFrame(ctx);
         writer.open();
         indexHelper.open();
         IIndex index = indexHelper.getIndexInstance();
@@ -134,8 +136,9 @@ public class IndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryIn
             }
         }
         // Pass a copy of the frame to next op.
-        System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
-        FrameUtils.flushFrame(writeBuffer, writer);
+        writeBuffer.ensureFrameSize(buffer.capacity());
+        FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+        FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index a3f4e6f..fd727cf 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -18,6 +18,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -45,7 +46,6 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
     protected final IIndexDataflowHelper indexHelper;
     protected FrameTupleAccessor accessor;
 
-    protected ByteBuffer writeBuffer;
     protected FrameTupleAppender appender;
     protected ArrayTupleBuilder tb;
     protected DataOutput dos;
@@ -103,7 +103,7 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
 
     @Override
     public void open() throws HyracksDataException {
-        accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
+        accessor = new FrameTupleAccessor(inputRecDesc);
         writer.open();
         indexHelper.open();
         index = indexHelper.getIndexInstance();
@@ -126,11 +126,9 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
 
         try {
             searchPred = createSearchPredicate();
-            writeBuffer = ctx.allocateFrame();
             tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
             dos = tb.getDataOutput();
-            appender = new FrameTupleAppender(ctx.getFrameSize());
-            appender.reset(writeBuffer, true);
+            appender = new FrameTupleAppender(new VSizeFrame(ctx), true);
             ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
                     .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
             indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, searchCallback);
@@ -162,27 +160,13 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
                 dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
                 tb.addFieldEndOffset();
             }
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                FrameUtils.flushFrame(writeBuffer, writer);
-                appender.reset(writeBuffer, true);
-                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
-                            + appender.getBuffer().capacity() + ")");
-                }
-            }
+            FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+                    tb.getSize());
         }
 
         if (!matched && retainInput && retainNull) {
-            if (!appender.appendConcat(accessor, tupleIndex, nullTupleBuild.getFieldEndOffsets(),
-                    nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
-                FrameUtils.flushFrame(writeBuffer, writer);
-                appender.reset(writeBuffer, true);
-                if (!appender.appendConcat(accessor, tupleIndex, nullTupleBuild.getFieldEndOffsets(),
-                        nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
-                    throw new HyracksDataException("Record size larger than frame size ("
-                            + appender.getBuffer().capacity() + ")");
-                }
-            }
+            FrameUtils.appendConcatToWriter(writer, appender, accessor, tupleIndex,
+                    nullTupleBuild.getFieldEndOffsets(), nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize());
         }
     }
 
@@ -205,9 +189,7 @@ public abstract class IndexSearchOperatorNodePushable extends AbstractUnaryInput
     @Override
     public void close() throws HyracksDataException {
         try {
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(writeBuffer, writer);
-            }
+            appender.flush(writer, true);
             try {
                 cursor.close();
             } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index cb6270f..0db195d 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -15,8 +15,8 @@
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import java.io.DataOutput;
-import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -60,9 +60,7 @@ public class TreeIndexDiskOrderScanOperatorNodePushable extends AbstractUnaryOut
             try {
                 indexAccessor.diskOrderScan(cursor);
                 int fieldCount = treeIndex.getFieldCount();
-                ByteBuffer frame = ctx.allocateFrame();
-                FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-                appender.reset(frame, true);
+                FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
                 ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
                 DataOutput dos = tb.getDataOutput();
 
@@ -72,21 +70,15 @@ public class TreeIndexDiskOrderScanOperatorNodePushable extends AbstractUnaryOut
 
                     ITupleReference frameTuple = cursor.getTuple();
                     for (int i = 0; i < frameTuple.getFieldCount(); i++) {
-                        dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
+                        dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i),
+                                frameTuple.getFieldLength(i));
                         tb.addFieldEndOffset();
                     }
 
-                    if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                        FrameUtils.flushFrame(frame, writer);
-                        appender.reset(frame, true);
-                        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                            throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
-                        }
-                    }
-                }
-                if (appender.getTupleCount() > 0) {
-                    FrameUtils.flushFrame(frame, writer);
+                    FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+                            tb.getSize());
                 }
+                appender.flush(writer, true);
             } catch (Exception e) {
                 writer.fail();
                 throw new HyracksDataException(e);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
index 717d326..f340142 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
@@ -15,14 +15,13 @@
 package edu.uci.ics.hyracks.storage.am.common.dataflow;
 
 import java.io.DataOutput;
-import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 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.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
@@ -70,18 +69,18 @@ public class TreeIndexStatsOperatorNodePushable extends AbstractUnaryOutputSourc
                     .getInteriorFrameFactory().createFrame(), treeIndex.getFreePageManager().getMetaDataFrameFactory()
                     .createFrame());
             // Write the stats output as a single string field.
-            ByteBuffer frame = ctx.allocateFrame();
-            FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-            appender.reset(frame, true);
+            FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
             ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
             DataOutput dos = tb.getDataOutput();
             tb.reset();
             UTF8StringSerializerDeserializer.INSTANCE.serialize(stats.toString(), dos);
             tb.addFieldEndOffset();
             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
+                throw new HyracksDataException(
+                        "Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity()
+                                + ")");
             }
-            FrameUtils.flushFrame(frame, writer);
+            appender.flush(writer, false);
         } catch (Exception e) {
             writer.fail();
         } finally {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
index 5bf52e4..fd693cf 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -30,6 +30,12 @@ public class LSMIndexInsertUpdateDeleteOperatorNodePushable extends IndexInsertU
 
     protected FrameTupleAppender appender;
 
+    @Override
+    public void open() throws HyracksDataException {
+        super.open();
+        appender = new FrameTupleAppender(writeBuffer);
+    }
+
     public LSMIndexInsertUpdateDeleteOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
             int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider, IndexOperation op) {
         super(opDesc, ctx, partition, fieldPermutation, recordDescProvider, op);
@@ -85,8 +91,8 @@ public class LSMIndexInsertUpdateDeleteOperatorNodePushable extends IndexInsertU
                         break;
                     }
                     default: {
-                        throw new HyracksDataException("Unsupported operation " + op
-                                + " in tree index InsertUpdateDelete operator");
+                        throw new HyracksDataException(
+                                "Unsupported operation " + op + " in tree index InsertUpdateDelete operator");
                     }
                 }
             } catch (HyracksDataException e) {
@@ -97,8 +103,9 @@ public class LSMIndexInsertUpdateDeleteOperatorNodePushable extends IndexInsertU
         }
         if (nextFlushTupleIndex == 0) {
             // No partial flushing was necessary. Forward entire frame.
-            System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
-            FrameUtils.flushFrame(writeBuffer, writer);
+            writeBuffer.ensureFrameSize(buffer.capacity());
+            FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+            FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
         } else {
             // Flush remaining partial frame.
             flushPartialFrame(nextFlushTupleIndex, tupleCount);
@@ -106,17 +113,9 @@ public class LSMIndexInsertUpdateDeleteOperatorNodePushable extends IndexInsertU
     }
 
     private void flushPartialFrame(int startTupleIndex, int endTupleIndex) throws HyracksDataException {
-        if (appender == null) {
-            appender = new FrameTupleAppender(ctx.getFrameSize());
-        }
-        appender.reset(writeBuffer, true);
         for (int i = startTupleIndex; i < endTupleIndex; i++) {
-            if (!appender.append(accessor, i)) {
-                throw new HyracksDataException("Record size ("
-                        + (accessor.getTupleEndOffset(i) - accessor.getTupleStartOffset(i))
-                        + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
-            }
+            FrameUtils.appendToWriter(writer, appender, accessor, i);
         }
-        FrameUtils.flushFrame(writeBuffer, writer);
+        appender.flush(writer, true);
     }
 }


[13/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksFrameMgrContext.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksFrameMgrContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksFrameMgrContext.java
new file mode 100644
index 0000000..d7a18bc
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksFrameMgrContext.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.context;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IHyracksFrameMgrContext {
+    int getInitialFrameSize();
+
+    //TODO tobedeleted
+    ByteBuffer allocateFrame() throws HyracksDataException;
+
+    ByteBuffer allocateFrame(int bytes) throws HyracksDataException;
+
+    ByteBuffer reallocateFrame(ByteBuffer tobeDeallocate, int newSizeInBytes, boolean copyOldData)
+            throws HyracksDataException;
+
+    /**
+     * The caller should call this method to return the pre-allocated frames.
+     *
+     * @param bytes
+     */
+    void deallocateFrames(int bytes);
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
index d2c2ee3..dec0bfd 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
@@ -14,13 +14,12 @@
  */
 package edu.uci.ics.hyracks.api.dataset;
 
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IHyracksDatasetReader {
     public Status getResultStatus();
 
-    public int read(ByteBuffer buffer) throws HyracksDataException;
+    public int read(IFrame frame) throws HyracksDataException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
index 97ce664..9975fe3 100644
--- a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
@@ -14,36 +14,21 @@
  */
 package edu.uci.ics.hyracks.client.dataset;
 
-import java.nio.ByteBuffer;
-
 import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
 import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
 
-public class DatasetClientContext implements IHyracksCommonContext {
+public class DatasetClientContext extends FrameManager implements IHyracksCommonContext {
     private final int frameSize;
 
     public DatasetClientContext(int frameSize) {
+        super(frameSize);
         this.frameSize = frameSize;
     }
 
     @Override
-    public int getFrameSize() {
-        return frameSize;
-    }
-
-    @Override
     public IIOManager getIOManager() {
         return null;
     }
 
-    @Override
-    public ByteBuffer allocateFrame() {
-        return ByteBuffer.allocate(frameSize);
-    }
-
-    @Override
-    public void deallocateFrames(int frameCount) {
-        // TODO Auto-generated method stub
-        
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
index 062b5bf..34c4d6e 100644
--- a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
@@ -24,6 +24,8 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.NetworkAddress;
 import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
 import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
@@ -64,7 +66,8 @@ public class HyracksDatasetReader implements IHyracksDatasetReader {
     private static int NUM_READ_BUFFERS = 1;
 
     public HyracksDatasetReader(IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection,
-            ClientNetworkManager netManager, DatasetClientContext datasetClientCtx, JobId jobId, ResultSetId resultSetId)
+            ClientNetworkManager netManager, DatasetClientContext datasetClientCtx, JobId jobId,
+            ResultSetId resultSetId)
             throws Exception {
         this.datasetDirectoryServiceConnection = datasetDirectoryServiceConnection;
         this.netManager = netManager;
@@ -119,7 +122,8 @@ public class HyracksDatasetReader implements IHyracksDatasetReader {
     }
 
     @Override
-    public int read(ByteBuffer buffer) throws HyracksDataException {
+    public int read(IFrame frame) throws HyracksDataException {
+        frame.reset();
         ByteBuffer readBuffer;
         int readSize = 0;
 
@@ -129,7 +133,7 @@ public class HyracksDatasetReader implements IHyracksDatasetReader {
             }
         }
 
-        while (readSize <= 0
+        while (readSize < frame.getFrameSize()
                 && !((lastReadPartition == knownRecords.length - 1) && isPartitionReadComplete(lastMonitor))) {
             waitForNextFrame(lastMonitor);
             if (isPartitionReadComplete(lastMonitor)) {
@@ -142,14 +146,23 @@ public class HyracksDatasetReader implements IHyracksDatasetReader {
                 readBuffer = resultChannel.getNextBuffer();
                 lastMonitor.notifyFrameRead();
                 if (readBuffer != null) {
-                    buffer.put(readBuffer);
-                    buffer.flip();
-                    readSize = buffer.limit();
-                    resultChannel.recycleBuffer(readBuffer);
+                    if (readSize <=0) {
+                        int nBlocks = FrameHelper.deserializeNumOfMinFrame(readBuffer);
+                        frame.ensureFrameSize(frame.getMinSize() * nBlocks);
+                        frame.getBuffer().clear();
+                        frame.getBuffer().put(readBuffer);
+                        resultChannel.recycleBuffer(readBuffer);
+                        readSize = frame.getBuffer().position();
+                    } else {
+                        frame.getBuffer().put(readBuffer);
+                        resultChannel.recycleBuffer(readBuffer);
+                        readSize = frame.getBuffer().position();
+                    }
                 }
             }
         }
 
+        frame.getBuffer().flip();
         return readSize;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
index c5cb7d0..87fa23c 100644
--- a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
@@ -95,7 +95,7 @@ public class NetworkInputChannel implements IInputChannel {
         }
         ccb.getReadInterface().setFullBufferAcceptor(new ReadFullBufferAcceptor());
         ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
-        ccb.getReadInterface().setBufferFactory(new ReadBufferFactory(nBuffers, ctx), nBuffers, ctx.getFrameSize());
+        ccb.getReadInterface().setBufferFactory(new ReadBufferFactory(nBuffers, ctx), nBuffers, ctx.getInitialFrameSize());
         ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
         writeBuffer.putLong(partitionId.getJobId().getId());
         writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
index b573b73..5ac0a47 100644
--- a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
@@ -54,32 +54,36 @@ public class NetworkOutputChannel implements IFrameWriter {
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
         ByteBuffer destBuffer = null;
-        synchronized (this) {
-            while (true) {
-                if (aborted) {
-                    throw new HyracksDataException("Connection has been aborted");
-                }
-                destBuffer = emptyStack.poll();
-                if (destBuffer == null && allocateCounter < nBuffers) {
-                    destBuffer = ByteBuffer.allocateDirect(frameSize);
-                    allocateCounter++;
-                }
-                if (destBuffer != null) {
-                    break;
-                }
-                try {
-                    wait();
-                } catch (InterruptedException e) {
-                    throw new HyracksDataException(e);
+        int startPos = 0;
+        do {
+            synchronized (this) {
+                while (true) {
+                    if (aborted) {
+                        throw new HyracksDataException("Connection has been aborted");
+                    }
+                    destBuffer = emptyStack.poll();
+                    if (destBuffer == null && allocateCounter < nBuffers) {
+                        destBuffer = ByteBuffer.allocateDirect(frameSize);
+                        allocateCounter++;
+                    }
+                    if (destBuffer != null) {
+                        break;
+                    }
+                    try {
+                        wait();
+                    } catch (InterruptedException e) {
+                        throw new HyracksDataException(e);
+                    }
                 }
             }
-        }
-        buffer.position(0);
-        buffer.limit(destBuffer.capacity());
-        destBuffer.clear();
-        destBuffer.put(buffer);
-        destBuffer.flip();
-        ccb.getWriteInterface().getFullBufferAcceptor().accept(destBuffer);
+            buffer.position(startPos);
+            startPos = Math.min(startPos + destBuffer.capacity(), buffer.capacity());
+            buffer.limit(startPos);
+            destBuffer.clear();
+            destBuffer.put(buffer);
+            destBuffer.flip();
+            ccb.getWriteInterface().getFullBufferAcceptor().accept(destBuffer);
+        } while (startPos < buffer.capacity());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java
index c59398c..641b7bf 100644
--- a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java
@@ -30,7 +30,7 @@ public class ReadBufferFactory implements IBufferFactory {
 
     public ReadBufferFactory(int limit, IHyracksCommonContext ctx) {
         this.limit = limit;
-        this.frameSize = ctx.getFrameSize();
+        this.frameSize = ctx.getInitialFrameSize();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 89c5b75..c6e49e1 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -56,6 +56,7 @@ import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
 import edu.uci.ics.hyracks.control.nc.io.IOManager;
 import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
 import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
 
 public class Joblet implements IHyracksJobletContext, ICounterContext {
     private static final Logger LOGGER = Logger.getLogger(Joblet.class.getName());
@@ -88,7 +89,7 @@ public class Joblet implements IHyracksJobletContext, ICounterContext {
 
     private final IJobletEventListener jobletEventListener;
 
-    private final int frameSize;
+    private final FrameManager frameManager;
 
     private final AtomicLong memoryAllocation;
 
@@ -102,7 +103,7 @@ public class Joblet implements IHyracksJobletContext, ICounterContext {
         this.appCtx = appCtx;
         this.deploymentId = deploymentId;
         this.jobId = jobId;
-        this.frameSize = acg.getFrameSize();
+        this.frameManager = new FrameManager(acg.getFrameSize());
         memoryAllocation = new AtomicLong();
         this.acg = acg;
         partitionRequestMap = new HashMap<PartitionId, IPartitionCollector>();
@@ -222,23 +223,33 @@ public class Joblet implements IHyracksJobletContext, ICounterContext {
     }
 
     ByteBuffer allocateFrame() throws HyracksDataException {
-        if (appCtx.getMemoryManager().allocate(frameSize)) {
-            memoryAllocation.addAndGet(frameSize);
-            return ByteBuffer.allocate(frameSize);
-        }
+        return frameManager.allocateFrame();
+    }
+
+    ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+        if (appCtx.getMemoryManager().allocate(bytes)) {
+            memoryAllocation.addAndGet(bytes);
+            return frameManager.allocateFrame(bytes);
+       }
         throw new HyracksDataException("Unable to allocate frame: Not enough memory");
     }
 
-    public void deallocateFrames(int nFrames) {
-        memoryAllocation.addAndGet(nFrames * frameSize);
-        appCtx.getMemoryManager().deallocate(nFrames * frameSize);
+    ByteBuffer reallocateFrame(ByteBuffer usedBuffer, int newFrameSizeInBytes, boolean copyOldData)
+            throws HyracksDataException {
+        return frameManager.reallocateFrame(usedBuffer, newFrameSizeInBytes, copyOldData);
+    }
+
+    void deallocateFrames(int bytes) {
+        memoryAllocation.addAndGet(bytes);
+        appCtx.getMemoryManager().deallocate(bytes);
+        frameManager.deallocateFrames(bytes);
     }
 
-    final int getFrameSize() {
-        return frameSize;
+    public final int getFrameSize() {
+        return frameManager.getInitialFrameSize();
     }
 
-    IIOManager getIOManager() {
+    public IIOManager getIOManager() {
         return appCtx.getRootContext().getIOManager();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index 3014024..387ab04 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -25,6 +25,7 @@ import java.util.Set;
 import java.util.concurrent.Executor;
 import java.util.concurrent.Semaphore;
 
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
@@ -120,12 +121,23 @@ public class Task implements IHyracksTaskContext, ICounterContext, Runnable {
     }
 
     @Override
-    public void deallocateFrames(int frameCount) {
-        joblet.deallocateFrames(frameCount);
+    public ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+        return joblet.allocateFrame(bytes);
     }
 
     @Override
-    public int getFrameSize() {
+    public ByteBuffer reallocateFrame(ByteBuffer usedBuffer, int newSizeInBytes, boolean copyOldData)
+            throws HyracksDataException {
+        return joblet.reallocateFrame(usedBuffer, newSizeInBytes, copyOldData);
+    }
+
+    @Override
+    public void deallocateFrames(int bytes) {
+        joblet.deallocateFrames(bytes);
+    }
+
+    @Override
+    public int getInitialFrameSize() {
         return joblet.getFrameSize();
     }
 
@@ -317,12 +329,12 @@ public class Task implements IHyracksTaskContext, ICounterContext, Runnable {
                 try {
                     writer.open();
                     try {
-                        ByteBuffer buffer = allocateFrame();
-                        while (reader.nextFrame(buffer)) {
+                        VSizeFrame frame = new VSizeFrame(this);
+                        while( reader.nextFrame(frame)){
                             if (aborted) {
                                 return;
                             }
-                            buffer.flip();
+                            ByteBuffer buffer = frame.getBuffer();
                             writer.nextFrame(buffer);
                             buffer.compact();
                         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
index d3a6fb5..673a319 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
@@ -67,7 +67,7 @@ public class DatasetPartitionWriter implements IFrameWriter {
 
         resultSetPartitionId = new ResultSetPartitionId(jobId, rsId, partition);
         resultState = new ResultState(resultSetPartitionId, asyncMode, ctx.getIOManager(), fileFactory,
-                ctx.getFrameSize());
+                ctx.getInitialFrameSize());
     }
 
     public ResultState getResultState() {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
index c24034d..9ea6ddb 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
@@ -190,6 +190,9 @@ public class ResultState implements IStateObject {
                     initReadFileHandle();
                 }
                 readSize = ioManager.syncRead(readFileHandle, offset, buffer);
+                if (readSize < 0){
+                    throw new HyracksDataException("Premature end of file");
+                }
             }
 
             if (readSize < buffer.capacity()) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java
index 239b6aa..b3f3e2a 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java
@@ -103,6 +103,15 @@ public class IOManager implements IIOManager {
         }
     }
 
+    /**
+     * Please do check the return value of this read!
+     *
+     * @param fHandle
+     * @param offset
+     * @param data
+     * @return The number of bytes read, possibly zero, or -1 if the given offset is greater than or equal to the file's current size
+     * @throws HyracksDataException
+     */
     @Override
     public int syncRead(IFileHandle fHandle, long offset, ByteBuffer data) throws HyracksDataException {
         try {
@@ -111,7 +120,7 @@ public class IOManager implements IIOManager {
             while (remaining > 0) {
                 int len = ((FileHandle) fHandle).getFileChannel().read(data, offset);
                 if (len < 0) {
-                    return -1;
+                    return n == 0 ? -1 : n;
                 }
                 remaining -= len;
                 offset += len;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
index 816e345..b52a8a1 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
@@ -114,6 +114,9 @@ public class MaterializedPartitionInputChannel implements IInputChannel {
                 ByteBuffer destFrame = emptyQueue.poll();
                 buffer.position(0);
                 buffer.limit(buffer.capacity());
+                if (destFrame.capacity() < buffer.capacity()){
+                    throw new HyracksDataException("should never happen");
+                }
                 destFrame.clear();
                 destFrame.put(buffer);
                 fullQueue.add(destFrame);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
index b209cc1..4f5d60f 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
@@ -64,7 +64,7 @@ public class PartitionManager {
              */
             NetworkOutputChannel writer = partitionRequests.remove(pid);
             if (writer != null) {
-                writer.setFrameSize(partition.getTaskContext().getFrameSize());
+                writer.setFrameSize(partition.getTaskContext().getInitialFrameSize());
                 partition.writeTo(writer);
                 if (!partition.isReusable()) {
                     return;
@@ -116,7 +116,7 @@ public class PartitionManager {
             List<IPartition> pList = availablePartitionMap.get(partitionId);
             if (pList != null && !pList.isEmpty()) {
                 IPartition partition = pList.get(0);
-                writer.setFrameSize(partition.getTaskContext().getFrameSize());
+                writer.setFrameSize(partition.getTaskContext().getInitialFrameSize());
                 partition.writeTo(writer);
                 if (!partition.isReusable()) {
                     availablePartitionMap.remove(partitionId);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java
index e7102e1..c9fc237 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java
@@ -14,8 +14,7 @@
  */
 package edu.uci.ics.hyracks.control.nc.profiling;
 
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
@@ -44,8 +43,8 @@ public class ConnectorReceiverProfilingFrameReader implements IFrameReader {
     }
 
     @Override
-    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        boolean status = reader.nextFrame(buffer);
+    public boolean nextFrame(IFrame frame) throws HyracksDataException {
+        boolean status = reader.nextFrame(frame);
         if (status) {
             frameCounter.update(1);
         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/FrameManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/FrameManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/FrameManager.java
new file mode 100644
index 0000000..fd71d8b
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/FrameManager.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.control.nc.resources.memory;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.context.IHyracksFrameMgrContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FrameManager implements IHyracksFrameMgrContext {
+
+    private final int minFrameSize;
+
+    public FrameManager(int minFrameSize) {
+        this.minFrameSize = minFrameSize;
+    }
+
+    @Override
+    public int getInitialFrameSize() {
+        return minFrameSize;
+    }
+
+    @Override
+    public ByteBuffer allocateFrame() throws HyracksDataException {
+        return allocateFrame(minFrameSize);
+    }
+
+    @Override
+    public ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+        if (bytes % minFrameSize != 0) {
+            throw new HyracksDataException("The size should be an integral multiple of the default frame size");
+        }
+        ByteBuffer buffer = ByteBuffer.allocate(bytes);
+        if (bytes / minFrameSize > FrameConstants.MAX_NUM_MINFRAME) {
+            throw new HyracksDataException(
+                    "Unable to allocate frame larger than:" + FrameConstants.MAX_NUM_MINFRAME + " bytes");
+        }
+        FrameHelper.serializeFrameSize(buffer, (byte) (bytes / minFrameSize));
+        return (ByteBuffer) buffer.clear();
+    }
+
+    @Override
+    public ByteBuffer reallocateFrame(ByteBuffer tobeDeallocate, int newSizeInBytes, boolean copyOldData)
+            throws HyracksDataException {
+        if (!copyOldData) {
+            deallocateFrames(tobeDeallocate.capacity());
+            return allocateFrame(newSizeInBytes);
+        } else {
+            ByteBuffer buffer = allocateFrame(newSizeInBytes);
+            int limit = Math.min(newSizeInBytes, tobeDeallocate.capacity());
+            int pos = Math.min(limit, tobeDeallocate.position());
+            tobeDeallocate.position(0);
+            tobeDeallocate.limit(limit);
+            buffer.put(tobeDeallocate);
+            buffer.position(pos);
+
+            if (newSizeInBytes / minFrameSize > FrameConstants.MAX_NUM_MINFRAME) {
+                throw new HyracksDataException("Unable to allocate frame of size bigger than MinFrameSize * "
+                        + FrameConstants.MAX_NUM_MINFRAME);
+            }
+            FrameHelper.serializeFrameSize(buffer, (byte) (newSizeInBytes / minFrameSize));
+            return buffer;
+        }
+    }
+
+    @Override
+    public void deallocateFrames(int bytes) {
+        //TODO make a global memory manager to allocate and deallocate the frames.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/pom.xml b/hyracks/hyracks-dataflow-common/pom.xml
index 9f965a3..3f2936d 100644
--- a/hyracks/hyracks-dataflow-common/pom.xml
+++ b/hyracks/hyracks-dataflow-common/pom.xml
@@ -57,6 +57,12 @@
   		<groupId>edu.uci.ics.hyracks</groupId>
   		<artifactId>hyracks-data-std</artifactId>
   		<version>0.2.16-SNAPSHOT</version>
+    </dependency>
+   	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-control-nc</artifactId>
+  		<version>0.2.16-SNAPSHOT</version>
+        <scope>test</scope>
   	</dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java
new file mode 100644
index 0000000..1f501aa
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+
+public class AbstractFrameAppender implements IFrameAppender {
+    protected IFrame frame;
+    protected byte[] array; // cached the getBuffer().array to speed up byte array access a little
+
+    protected int tupleCount;
+    protected int tupleDataEndOffset;
+
+    @Override
+    public void reset(IFrame frame, boolean clear) throws HyracksDataException {
+        this.frame = frame;
+        if (clear) {
+            this.frame.reset();
+        }
+        reset(getBuffer(), clear);
+    }
+
+    protected boolean hasEnoughSpace(int fieldCount, int tupleLength) {
+        return tupleDataEndOffset + FrameHelper.calcSpaceInFrame(fieldCount, tupleLength)
+                + tupleCount * FrameConstants.SIZE_LEN
+                <= FrameHelper.getTupleCountOffset(frame.getFrameSize());
+    }
+
+    private void reset(ByteBuffer buffer, boolean clear) {
+        array = buffer.array();
+        if (clear) {
+            IntSerDeUtils.putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), 0);
+            tupleCount = 0;
+            tupleDataEndOffset = FrameConstants.TUPLE_START_OFFSET;
+        } else {
+            tupleCount = IntSerDeUtils.getInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()));
+            tupleDataEndOffset = tupleCount == 0 ?
+                    FrameConstants.TUPLE_START_OFFSET :
+                    IntSerDeUtils.getInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize())
+                            - tupleCount * FrameConstants.SIZE_LEN);
+        }
+    }
+
+    @Override
+    public int getTupleCount() {
+        return tupleCount;
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return frame.getBuffer();
+    }
+
+    @Override
+    public void flush(IFrameWriter outWriter, boolean clearFrame) throws HyracksDataException {
+        getBuffer().clear();
+        if (getTupleCount() > 0) {
+            outWriter.nextFrame(getBuffer());
+        }
+        if (clearFrame) {
+            frame.reset();
+            reset(getBuffer(), true);
+        }
+    }
+
+    protected boolean canHoldNewTuple(int fieldCount, int dataLength) throws HyracksDataException {
+        if (hasEnoughSpace(fieldCount, dataLength)) {
+            return true;
+        }
+        if (tupleCount == 0) {
+            frame.ensureFrameSize(FrameHelper.calcAlignedFrameSizeToStore(fieldCount, dataLength, frame.getMinSize()));
+            reset(frame.getBuffer(), true);
+            return true;
+        }
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
index 9223740..53c5eb3 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
@@ -17,6 +17,7 @@ package edu.uci.ics.hyracks.dataflow.common.comm.io;
 import java.io.DataOutput;
 import java.io.IOException;
 
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameConstants.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameConstants.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameConstants.java
deleted file mode 100644
index 43538d7..0000000
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameConstants.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.common.comm.io;
-
-public interface FrameConstants {
-    public static final int SIZE_LEN = 4;
-
-    public static final boolean DEBUG_FRAME_IO = false;
-
-    public static final int FRAME_FIELD_MAGIC = 0x12345678;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
index 556ecbb..b1052e7 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
@@ -20,6 +20,7 @@ import java.nio.ByteBuffer;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
 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.util.ByteBufferInputStream;
@@ -41,11 +42,11 @@ public class FrameDeserializer {
 
     private ByteBuffer buffer;
 
-    public FrameDeserializer(int frameSize, RecordDescriptor recordDescriptor) {
+    public FrameDeserializer(RecordDescriptor recordDescriptor) {
         this.bbis = new ByteBufferInputStream();
         this.di = new DataInputStream(bbis);
         this.recordDescriptor = recordDescriptor;
-        frameTupleAccessor = new FrameTupleAccessor(frameSize, recordDescriptor);
+        frameTupleAccessor = new FrameTupleAccessor(recordDescriptor);
     }
 
     public void reset(ByteBuffer buffer) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
index b94a219..2dcf84c 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
@@ -14,16 +14,16 @@
  */
 package edu.uci.ics.hyracks.dataflow.common.comm.io;
 
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public class FrameDeserializingDataReader implements IOpenableDataReader<Object[]> {
-    private final ByteBuffer buffer;
+    private final IFrame frame;
 
     private boolean eos;
 
@@ -35,16 +35,15 @@ public class FrameDeserializingDataReader implements IOpenableDataReader<Object[
 
     public FrameDeserializingDataReader(IHyracksTaskContext ctx, IFrameReader frameReader,
             RecordDescriptor recordDescriptor) throws HyracksDataException {
-        buffer = ctx.allocateFrame();
+        this.frame = new VSizeFrame(ctx);
         this.frameReader = frameReader;
-        this.frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), recordDescriptor);
+        this.frameDeserializer = new FrameDeserializer(recordDescriptor);
     }
 
     @Override
     public void open() throws HyracksDataException {
         frameReader.open();
-        buffer.clear();
-        buffer.flip();
+        frame.reset();
         eos = false;
         first = true;
     }
@@ -64,11 +63,11 @@ public class FrameDeserializingDataReader implements IOpenableDataReader<Object[
             if (!first && !frameDeserializer.done()) {
                 return frameDeserializer.deserializeRecord();
             }
-            buffer.clear();
-            if (!frameReader.nextFrame(buffer)) {
+            frame.reset();
+            if (!frameReader.nextFrame(frame)) {
                 eos = true;
             } else {
-                frameDeserializer.reset(buffer);
+                frameDeserializer.reset(frame.getBuffer());
             }
             first = false;
         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java
index b3a72d7..8855de5 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java
@@ -29,7 +29,7 @@ public class FrameDeserializingDataWriter implements IFrameWriter {
     public FrameDeserializingDataWriter(IHyracksTaskContext ctx, IOpenableDataWriter<Object[]> writer,
             RecordDescriptor recordDescriptor) {
         this.writer = writer;
-        this.frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), recordDescriptor);
+        this.frameDeserializer = new FrameDeserializer(recordDescriptor);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldAppender.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldAppender.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldAppender.java
new file mode 100644
index 0000000..2db3e44
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldAppender.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameFieldAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+
+public class FrameFixedFieldAppender extends AbstractFrameAppender implements IFrameFieldAppender {
+    private final int fieldCount;
+    private int lastFieldEndOffset;
+    private int currentField;
+    private int leftOverSize;
+    private byte[] cachedLeftOverFields;
+
+    public FrameFixedFieldAppender(int numberFields) {
+        this.fieldCount = numberFields;
+        this.lastFieldEndOffset = 0;
+        this.currentField = 0;
+        this.leftOverSize = 0;
+    }
+
+    @Override
+    public void reset(IFrame frame, boolean clear) throws HyracksDataException {
+        super.reset(frame, clear);
+        lastFieldEndOffset = 0;
+        currentField = 0;
+        leftOverSize = 0;
+    }
+
+    @Override
+    public void flush(IFrameWriter outWriter, boolean clearFrame) throws HyracksDataException {
+        super.flush(outWriter, clearFrame);
+        if (clearFrame) {
+            if (leftOverSize > 0) {
+                if (!canHoldNewTuple(0, leftOverSize)) {
+                    throw new HyracksDataException(
+                            "The given frame can not be extended to insert the leftover data from the last record");
+                }
+                System.arraycopy(cachedLeftOverFields, 0, array, tupleDataEndOffset, leftOverSize);
+                leftOverSize = 0;
+            }
+        }
+    }
+
+    public boolean appendField(byte[] bytes, int offset, int length) throws HyracksDataException {
+        if (canHoldNewTuple(fieldCount, lastFieldEndOffset + length)) {
+            int currentFieldDataStart = tupleDataEndOffset + fieldCount * 4 + lastFieldEndOffset;
+            System.arraycopy(bytes, offset, array, currentFieldDataStart, length);
+            lastFieldEndOffset = lastFieldEndOffset + length;
+            IntSerDeUtils.putInt(array, tupleDataEndOffset + currentField * 4, lastFieldEndOffset);
+            if (++currentField == fieldCount) {
+                tupleDataEndOffset += fieldCount * 4 + lastFieldEndOffset;
+                IntSerDeUtils
+                        .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1),
+                                tupleDataEndOffset);
+                ++tupleCount;
+                IntSerDeUtils.putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
+
+                //reset for the next tuple
+                currentField = 0;
+                lastFieldEndOffset = 0;
+            }
+            return true;
+        } else {
+            if (currentField > 0) {
+                copyLeftOverData();
+            }
+            return false;
+        }
+    }
+
+    private void copyLeftOverData() {
+        leftOverSize = lastFieldEndOffset + fieldCount * 4;
+        if (cachedLeftOverFields == null || cachedLeftOverFields.length < leftOverSize) {
+            cachedLeftOverFields = new byte[leftOverSize];
+        }
+        System.arraycopy(array, tupleDataEndOffset, cachedLeftOverFields, 0, leftOverSize);
+    }
+
+    public boolean appendField(IFrameTupleAccessor fta, int tIndex, int fIndex) throws HyracksDataException {
+        int startOffset = fta.getTupleStartOffset(tIndex);
+        int fStartOffset = fta.getFieldStartOffset(tIndex, fIndex);
+        int fLen = fta.getFieldEndOffset(tIndex, fIndex) - fStartOffset;
+        return appendField(fta.getBuffer().array(), startOffset + fta.getFieldSlotsLength() + fStartOffset, fLen);
+    }
+
+    public boolean hasLeftOverFields() {
+        return currentField != 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldTupleAppender.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldTupleAppender.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldTupleAppender.java
new file mode 100644
index 0000000..74289c6
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldTupleAppender.java
@@ -0,0 +1,130 @@
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameFieldAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This appender can appendTuple and appendField but at the expense of additional checks.
+ * Please use this Field/Tuple mixed appender only if you don't know the sequence of append functions.
+ * Try using {@link FrameFixedFieldAppender} if you only want to appendFields.
+ * and using {@link FrameTupleAppender} if you only want to appendTuples.
+ */
+public class FrameFixedFieldTupleAppender implements IFrameTupleAppender, IFrameFieldAppender {
+
+    private FrameFixedFieldAppender fieldAppender;
+    private FrameTupleAppender tupleAppender;
+    private IFrame sharedFrame;
+    private IFrameAppender lastAppender;
+
+    public FrameFixedFieldTupleAppender(int numFields) {
+        tupleAppender = new FrameTupleAppender();
+        fieldAppender = new FrameFixedFieldAppender(numFields);
+        lastAppender = tupleAppender;
+    }
+
+    private void resetAppenderIfNecessary(IFrameAppender appender) throws HyracksDataException {
+        if (lastAppender != appender) {
+            if (lastAppender == fieldAppender) {
+                if (fieldAppender.hasLeftOverFields()) {
+                    throw new HyracksDataException("The previous appended fields haven't been flushed yet.");
+                }
+            }
+            appender.reset(sharedFrame, false);
+            lastAppender = appender;
+        }
+    }
+
+    @Override
+    public boolean appendField(byte[] bytes, int offset, int length) throws HyracksDataException {
+        resetAppenderIfNecessary(fieldAppender);
+        return fieldAppender.appendField(bytes, offset, length);
+    }
+
+    @Override
+    public boolean appendField(IFrameTupleAccessor accessor, int tid, int fid) throws HyracksDataException {
+        resetAppenderIfNecessary(fieldAppender);
+        return fieldAppender.appendField(accessor, tid, fid);
+    }
+
+    @Override
+    public boolean append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException {
+        resetAppenderIfNecessary(tupleAppender);
+        return tupleAppender.append(tupleAccessor, tIndex);
+    }
+
+    @Override
+    public boolean append(int[] fieldSlots, byte[] bytes, int offset, int length) throws HyracksDataException {
+        resetAppenderIfNecessary(tupleAppender);
+        return tupleAppender.append(fieldSlots, bytes, offset, length);
+    }
+
+    @Override
+    public boolean append(byte[] bytes, int offset, int length) throws HyracksDataException {
+        resetAppenderIfNecessary(tupleAppender);
+        return tupleAppender.append(bytes, offset, length);
+    }
+
+    @Override
+    public boolean appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length)
+            throws HyracksDataException {
+        resetAppenderIfNecessary(tupleAppender);
+        return tupleAppender.appendSkipEmptyField(fieldSlots, bytes, offset, length);
+    }
+
+    @Override
+    public boolean append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset)
+            throws HyracksDataException {
+        resetAppenderIfNecessary(tupleAppender);
+        return tupleAppender.append(tupleAccessor, tStartOffset, tEndOffset);
+    }
+
+    @Override
+    public boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
+            throws HyracksDataException {
+        resetAppenderIfNecessary(tupleAppender);
+        return tupleAppender.appendConcat(accessor0, tIndex0, accessor1, tIndex1);
+    }
+
+    @Override
+    public boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, int[] fieldSlots1, byte[] bytes1,
+            int offset1, int dataLen1) throws HyracksDataException {
+        resetAppenderIfNecessary(tupleAppender);
+        return tupleAppender.appendConcat(accessor0, tIndex0, fieldSlots1, bytes1, offset1, dataLen1);
+    }
+
+    @Override
+    public boolean appendProjection(IFrameTupleAccessor accessor, int tIndex, int[] fields)
+            throws HyracksDataException {
+        resetAppenderIfNecessary(tupleAppender);
+        return tupleAppender.appendProjection(accessor, tIndex, fields);
+    }
+
+    @Override
+    public void reset(IFrame frame, boolean clear) throws HyracksDataException {
+        sharedFrame = frame;
+        tupleAppender.reset(sharedFrame, clear);
+        fieldAppender.reset(sharedFrame, clear);
+    }
+
+    @Override
+    public int getTupleCount() {
+        return lastAppender.getTupleCount();
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return lastAppender.getBuffer();
+    }
+
+    @Override
+    public void flush(IFrameWriter outWriter, boolean clear) throws HyracksDataException {
+        lastAppender.flush(outWriter, clear);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
index 1810674..90f3d25 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
@@ -14,28 +14,26 @@
  */
 package edu.uci.ics.hyracks.dataflow.common.comm.io;
 
-import java.nio.ByteBuffer;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 
 public class FrameOutputStream extends ByteArrayAccessibleOutputStream {
     private static final Logger LOGGER = Logger.getLogger(FrameOutputStream.class.getName());
 
     private final FrameTupleAppender frameTupleAppender;
 
-    public FrameOutputStream(int frameSize) {
-        super(frameSize);
-        this.frameTupleAppender = new FrameTupleAppender(frameSize);
+    public FrameOutputStream(int initialStreamCapaciy) {
+        super(initialStreamCapaciy);
+        this.frameTupleAppender = new FrameTupleAppender();
     }
 
-    public void reset(ByteBuffer buffer, boolean clear) {
-        if (clear) {
-            buffer.clear();
-        }
-        frameTupleAppender.reset(buffer, clear);
+    public void reset(IFrame frame, boolean clear) throws HyracksDataException {
+        frameTupleAppender.reset(frame, clear);
     }
 
     public int getTupleCount() {
@@ -46,7 +44,7 @@ public class FrameOutputStream extends ByteArrayAccessibleOutputStream {
         return tupleCount;
     }
 
-    public boolean appendTuple() {
+    public boolean appendTuple() throws HyracksDataException {
         if (LOGGER.isLoggable(Level.FINEST)) {
             LOGGER.finest("appendTuple(): tuple size: " + count);
         }
@@ -54,4 +52,8 @@ public class FrameOutputStream extends ByteArrayAccessibleOutputStream {
         count = 0;
         return appended;
     }
+
+    public void flush(IFrameWriter writer) throws HyracksDataException {
+        frameTupleAppender.flush(writer, true);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
index 6edd647..600e0b4 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
@@ -17,6 +17,7 @@ package edu.uci.ics.hyracks.dataflow.common.comm.io;
 import java.io.DataInputStream;
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
 import edu.uci.ics.hyracks.api.comm.FrameHelper;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -26,29 +27,33 @@ import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 
 /**
  * FrameTupleCursor is used to navigate over tuples in a Frame. A frame is
- * formatted with tuple data concatenated starting at offset 0, one tuple after
- * another. Offset FS - 4 holds an int indicating the number of tuples (N) in
+ * formatted with tuple data concatenated starting at offset 1, one tuple after
+ * another. The first byte is used to notify how big the frame is, so the maximum frame size is 255 * initialFrameSetting.
+ * Offset FS - 4 holds an int indicating the number of tuples (N) in
  * the frame. FS - ((i + 1) * 4) for i from 0 to N - 1 holds an int indicating
  * the offset of the (i + 1)^th tuple. Every tuple is organized as a sequence of
  * ints indicating the end of each field in the tuple relative to the end of the
  * field slots.
- *
- * @author vinayakb
  */
-public final class FrameTupleAccessor implements IFrameTupleAccessor {
-    private final int frameSize;
+public class FrameTupleAccessor implements IFrameTupleAccessor {
+    private int tupleCountOffset;
     private final RecordDescriptor recordDescriptor;
-
     private ByteBuffer buffer;
+    private int start;
 
-    public FrameTupleAccessor(int frameSize, RecordDescriptor recordDescriptor) {
-        this.frameSize = frameSize;
+    public FrameTupleAccessor(RecordDescriptor recordDescriptor) {
         this.recordDescriptor = recordDescriptor;
     }
 
     @Override
     public void reset(ByteBuffer buffer) {
+        reset(buffer, 0, buffer.limit());
+    }
+
+    public void reset(ByteBuffer buffer, int start, int length) {
         this.buffer = buffer;
+        this.start = start;
+        this.tupleCountOffset = start + FrameHelper.getTupleCountOffset(length);
     }
 
     @Override
@@ -58,28 +63,39 @@ public final class FrameTupleAccessor implements IFrameTupleAccessor {
 
     @Override
     public int getTupleCount() {
-        return IntSerDeUtils.getInt(buffer.array(), FrameHelper.getTupleCountOffset(frameSize));
+        return IntSerDeUtils.getInt(buffer.array(), tupleCountOffset);
     }
 
     @Override
     public int getTupleStartOffset(int tupleIndex) {
-        return tupleIndex == 0 ? 0 : IntSerDeUtils.getInt(buffer.array(), FrameHelper.getTupleCountOffset(frameSize)
-                - 4 * tupleIndex);
+        int offset = tupleIndex == 0 ?
+                FrameConstants.TUPLE_START_OFFSET :
+                IntSerDeUtils.getInt(buffer.array(), tupleCountOffset - 4 * tupleIndex);
+        return start + offset;
+    }
+
+    @Override
+    public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+        return getTupleStartOffset(tupleIndex) + getFieldSlotsLength() + getFieldStartOffset(tupleIndex, fIdx);
     }
 
     @Override
     public int getTupleEndOffset(int tupleIndex) {
-        return IntSerDeUtils.getInt(buffer.array(), FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
+        return start + IntSerDeUtils
+                .getInt(buffer.array(), tupleCountOffset - FrameConstants.SIZE_LEN * (tupleIndex + 1));
     }
 
     @Override
     public int getFieldStartOffset(int tupleIndex, int fIdx) {
-        return fIdx == 0 ? 0 : IntSerDeUtils.getInt(buffer.array(), getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+        return fIdx == 0 ?
+                0 :
+                IntSerDeUtils
+                        .getInt(buffer.array(), getTupleStartOffset(tupleIndex) + (fIdx - 1) * FrameConstants.SIZE_LEN);
     }
 
     @Override
     public int getFieldEndOffset(int tupleIndex, int fIdx) {
-        return IntSerDeUtils.getInt(buffer.array(), getTupleStartOffset(tupleIndex) + fIdx * 4);
+        return IntSerDeUtils.getInt(buffer.array(), getTupleStartOffset(tupleIndex) + fIdx * FrameConstants.SIZE_LEN);
     }
 
     @Override
@@ -88,34 +104,57 @@ public final class FrameTupleAccessor implements IFrameTupleAccessor {
     }
 
     @Override
+    public int getTupleLength(int tupleIndex) {
+        return getTupleEndOffset(tupleIndex) - getTupleStartOffset(tupleIndex);
+    }
+
+    @Override
     public int getFieldSlotsLength() {
-        return getFieldCount() * 4;
+        return getFieldCount() * FrameConstants.SIZE_LEN;
     }
 
-    public void prettyPrint() {
+    public void prettyPrint(String prefix) {
         ByteBufferInputStream bbis = new ByteBufferInputStream();
         DataInputStream dis = new DataInputStream(bbis);
         int tc = getTupleCount();
-        System.err.println("TC: " + tc);
+        StringBuilder sb = new StringBuilder();
+        sb.append(prefix).append("TC: " + tc).append("\n");
         for (int i = 0; i < tc; ++i) {
-            System.err.print(i + ":(" + getTupleStartOffset(i) + ", " + getTupleEndOffset(i) + ")[");
-            for (int j = 0; j < getFieldCount(); ++j) {
-                System.err.print(j + ":(" + getFieldStartOffset(i, j) + ", " + getFieldEndOffset(i, j) + ") ");
-                System.err.print("{");
-                bbis.setByteBuffer(buffer, getTupleStartOffset(i) + getFieldSlotsLength() + getFieldStartOffset(i, j));
-                try {
-                    System.err.print(recordDescriptor.getFields()[j].deserialize(dis));
-                } catch (HyracksDataException e) {
-                    e.printStackTrace();
-                }
-                System.err.print("}");
+            prettyPrint(i, bbis, dis, sb);
+        }
+        System.err.println(sb.toString());
+    }
+
+    public void prettyPrint() {
+        prettyPrint("");
+    }
+
+    protected void prettyPrint(int tid, ByteBufferInputStream bbis, DataInputStream dis, StringBuilder sb) {
+        sb.append(" tid" + tid + ":(" + getTupleStartOffset(tid) + ", " + getTupleEndOffset(tid) + ")[");
+        for (int j = 0; j < getFieldCount(); ++j) {
+            sb.append("f" + j + ":(" + getFieldStartOffset(tid, j) + ", " + getFieldEndOffset(tid, j) + ") ");
+            sb.append("{");
+            bbis.setByteBuffer(buffer, getTupleStartOffset(tid) + getFieldSlotsLength() + getFieldStartOffset(tid, j));
+            try {
+                sb.append(recordDescriptor.getFields()[j].deserialize(dis));
+            } catch (HyracksDataException e) {
+                e.printStackTrace();
             }
-            System.err.println("]");
+            sb.append("}");
         }
+        sb.append("\n");
+    }
+
+    public void prettyPrint(int tid) {
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+        DataInputStream dis = new DataInputStream(bbis);
+        StringBuilder sb = new StringBuilder();
+        prettyPrint(tid, bbis, dis, sb);
+        System.err.println(sb.toString());
     }
 
     @Override
     public int getFieldCount() {
         return recordDescriptor.getFieldCount();
     }
-}
\ No newline at end of file
+}


[10/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
index e695828..d0a1146 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
@@ -14,11 +14,6 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.group.sort;
 
-import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
@@ -31,20 +26,15 @@ import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupWriter;
 import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
-import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorterMergeSort;
-import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorterQuickSort;
-import edu.uci.ics.hyracks.dataflow.std.sort.IFrameSorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
 
 /**
  * Group-by aggregation is pushed before run file generation.
- * 
+ *
  * @author yingyib
  */
-public class ExternalSortGroupByRunGenerator implements IFrameWriter {
-    private final IHyracksTaskContext ctx;
-    private final IFrameSorter frameSorter;
-    private final List<IFrameReader> runs;
-    private final int maxSortFrames;
+public class ExternalSortGroupByRunGenerator extends ExternalSortRunGenerator {
 
     private final int[] groupFields;
     private final IBinaryComparatorFactory[] comparatorFactories;
@@ -52,86 +42,44 @@ public class ExternalSortGroupByRunGenerator implements IFrameWriter {
     private final RecordDescriptor inRecordDesc;
     private final RecordDescriptor outRecordDesc;
 
-    public ExternalSortGroupByRunGenerator(IHyracksTaskContext ctx, int[] sortFields, RecordDescriptor recordDesc,
+    public ExternalSortGroupByRunGenerator(IHyracksTaskContext ctx, int[] sortFields, RecordDescriptor inputRecordDesc,
             int framesLimit, int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
             IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory aggregatorFactory,
             RecordDescriptor outRecordDesc, Algorithm alg) throws HyracksDataException {
-        this.ctx = ctx;
-        if (alg == Algorithm.MERGE_SORT) {
-            frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
-                    recordDesc);
-        } else {
-            frameSorter = new FrameSorterQuickSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
-                    recordDesc);
-        }
-        this.runs = new LinkedList<IFrameReader>();
-        this.maxSortFrames = framesLimit - 1;
+        this(ctx, sortFields, inputRecordDesc, framesLimit, groupFields, firstKeyNormalizerFactory, comparatorFactories,
+                aggregatorFactory, outRecordDesc, alg, EnumFreeSlotPolicy.LAST_FIT);
+    }
+
+    public ExternalSortGroupByRunGenerator(IHyracksTaskContext ctx, int[] sortFields, RecordDescriptor inputRecordDesc,
+            int framesLimit, int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+            IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory aggregatorFactory,
+            RecordDescriptor outRecordDesc, Algorithm alg, EnumFreeSlotPolicy policy) throws HyracksDataException {
+        super(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, inputRecordDesc, alg, policy,
+                framesLimit);
+
         this.groupFields = groupFields;
         this.comparatorFactories = comparatorFactories;
         this.aggregatorFactory = aggregatorFactory;
-        this.inRecordDesc = recordDesc;
+        this.inRecordDesc = inputRecordDesc;
         this.outRecordDesc = outRecordDesc;
     }
 
     @Override
-    public void open() throws HyracksDataException {
-        runs.clear();
-        frameSorter.reset();
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        if (frameSorter.getFrameCount() >= maxSortFrames) {
-            flushFramesToRun();
-        }
-        frameSorter.insertFrame(buffer);
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        if (frameSorter.getFrameCount() > 0) {
-            if (runs.size() <= 0) {
-                frameSorter.sortFrames();
-            } else {
-                flushFramesToRun();
-            }
-        }
-    }
-
-    private void flushFramesToRun() throws HyracksDataException {
-        frameSorter.sortFrames();
+    protected RunFileWriter getRunFileWriter() throws HyracksDataException {
         FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
                 ExternalSortGroupByRunGenerator.class.getSimpleName());
-        RunFileWriter writer = new RunFileWriter(file, ctx.getIOManager());
+        return new RunFileWriter(file, ctx.getIOManager());
+    }
 
+    @Override
+    protected IFrameWriter getFlushableFrameWriter(RunFileWriter writer) throws HyracksDataException {
         //create group-by comparators
         IBinaryComparator[] comparators = new IBinaryComparator[Math
                 .min(groupFields.length, comparatorFactories.length)];
         for (int i = 0; i < comparators.length; i++) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
-        PreclusteredGroupWriter pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregatorFactory,
+        return new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregatorFactory,
                 this.inRecordDesc, this.outRecordDesc, writer, true);
-        pgw.open();
-
-        try {
-            frameSorter.flushFrames(pgw);
-        } finally {
-            pgw.close();
-        }
-        frameSorter.reset();
-        runs.add(writer.createReader());
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-    }
-
-    public IFrameSorter getFrameSorter() {
-        return frameSorter;
-    }
-
-    public List<IFrameReader> getRuns() {
-        return runs;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
index 2a580d3..7de400d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
@@ -14,12 +14,8 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.group.sort;
 
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.LinkedList;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
@@ -27,39 +23,25 @@ import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
 import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupWriter;
-import edu.uci.ics.hyracks.dataflow.std.sort.IFrameSorter;
-import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
+import edu.uci.ics.hyracks.dataflow.std.sort.ISorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
 
 /**
  * Group-by aggregation is pushed into multi-pass merge of external sort.
- * 
+ *
  * @author yingyib
  */
-public class ExternalSortGroupByRunMerger {
+public class ExternalSortGroupByRunMerger extends ExternalSortRunMerger {
 
-    private final IHyracksTaskContext ctx;
-    private final List<IFrameReader> runs;
     private final RecordDescriptor inputRecordDesc;
     private final RecordDescriptor partialAggRecordDesc;
     private final RecordDescriptor outRecordDesc;
-    private final int framesLimit;
-    private final IFrameWriter writer;
-    private List<ByteBuffer> inFrames;
-    private ByteBuffer outFrame;
-    private FrameTupleAppender outFrameAppender;
-
-    private final IFrameSorter frameSorter; // Used in External sort, no replacement
-    // selection
 
     private final int[] groupFields;
-    private final INormalizedKeyComputer firstKeyNkc;
-    private final IBinaryComparator[] comparators;
     private final IAggregatorDescriptorFactory mergeAggregatorFactory;
     private final IAggregatorDescriptorFactory partialAggregatorFactory;
     private final boolean localSide;
@@ -68,25 +50,19 @@ public class ExternalSortGroupByRunMerger {
     private final int[] mergeGroupFields;
     private final IBinaryComparator[] groupByComparators;
 
-    // Constructor for external sort, no replacement selection
-    public ExternalSortGroupByRunMerger(IHyracksTaskContext ctx, IFrameSorter frameSorter, List<IFrameReader> runs,
+    public ExternalSortGroupByRunMerger(IHyracksTaskContext ctx, ISorter frameSorter, List<RunAndMaxFrameSizePair> runs,
             int[] sortFields, RecordDescriptor inRecordDesc, RecordDescriptor partialAggRecordDesc,
             RecordDescriptor outRecordDesc, int framesLimit, IFrameWriter writer, int[] groupFields,
             INormalizedKeyComputer nmk, IBinaryComparator[] comparators,
             IAggregatorDescriptorFactory partialAggregatorFactory, IAggregatorDescriptorFactory aggregatorFactory,
             boolean localStage) {
-        this.ctx = ctx;
-        this.frameSorter = frameSorter;
-        this.runs = new LinkedList<IFrameReader>(runs);
+        super(ctx, frameSorter, runs, sortFields, comparators, nmk, partialAggRecordDesc, framesLimit,
+                writer);
         this.inputRecordDesc = inRecordDesc;
         this.partialAggRecordDesc = partialAggRecordDesc;
         this.outRecordDesc = outRecordDesc;
-        this.framesLimit = framesLimit;
-        this.writer = writer;
 
         this.groupFields = groupFields;
-        this.firstKeyNkc = nmk;
-        this.comparators = comparators;
         this.mergeAggregatorFactory = aggregatorFactory;
         this.partialAggregatorFactory = partialAggregatorFactory;
         this.localSide = localStage;
@@ -112,82 +88,38 @@ public class ExternalSortGroupByRunMerger {
         }
     }
 
-    public void process() throws HyracksDataException {
+    @Override
+    protected IFrameWriter prepareSkipMergingFinalResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
         IAggregatorDescriptorFactory aggregatorFactory = localSide ? partialAggregatorFactory : mergeAggregatorFactory;
-        PreclusteredGroupWriter pgw = new PreclusteredGroupWriter(ctx, groupFields, groupByComparators,
-                aggregatorFactory, inputRecordDesc, outRecordDesc, writer, false);
-        try {
-            if (runs.size() <= 0) {
-                pgw.open();
-                if (frameSorter != null && frameSorter.getFrameCount() > 0) {
-                    frameSorter.flushFrames(pgw);
-                }
-                /** recycle sort buffer */
-                frameSorter.close();
-            } else {
-                /** recycle sort buffer */
-                frameSorter.close();
-
-                inFrames = new ArrayList<ByteBuffer>();
-                outFrame = ctx.allocateFrame();
-                outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
-                outFrameAppender.reset(outFrame, true);
-                for (int i = 0; i < framesLimit - 1; ++i) {
-                    inFrames.add(ctx.allocateFrame());
-                }
-                int maxMergeWidth = framesLimit - 1;
-                while (runs.size() > maxMergeWidth) {
-                    int generationSeparator = 0;
-                    while (generationSeparator < runs.size() && runs.size() > maxMergeWidth) {
-                        int mergeWidth = Math.min(Math.min(runs.size() - generationSeparator, maxMergeWidth),
-                                runs.size() - maxMergeWidth + 1);
-                        FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortGroupByRunMerger.class
-                                .getSimpleName());
-                        IFrameWriter mergeResultWriter = new RunFileWriter(newRun, ctx.getIOManager());
-
-                        aggregatorFactory = localSide ? mergeAggregatorFactory : partialAggregatorFactory;
-                        pgw = new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators, aggregatorFactory,
-                                partialAggRecordDesc, partialAggRecordDesc, mergeResultWriter, true);
-                        pgw.open();
-
-                        IFrameReader[] runCursors = new RunFileReader[mergeWidth];
-                        for (int i = 0; i < mergeWidth; i++) {
-                            runCursors[i] = runs.get(generationSeparator + i);
-                        }
-                        merge(pgw, runCursors);
-                        pgw.close();
-                        runs.subList(generationSeparator, mergeWidth + generationSeparator).clear();
-                        runs.add(generationSeparator++, ((RunFileWriter) mergeResultWriter).createReader());
-                    }
-                }
-                if (!runs.isEmpty()) {
-                    pgw = new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators,
-                            mergeAggregatorFactory, partialAggRecordDesc, outRecordDesc, writer, false);
-                    pgw.open();
-                    IFrameReader[] runCursors = new RunFileReader[runs.size()];
-                    for (int i = 0; i < runCursors.length; i++) {
-                        runCursors[i] = runs.get(i);
-                    }
-                    merge(pgw, runCursors);
-                }
-            }
-        } catch (Exception e) {
-            pgw.fail();
-        } finally {
-            pgw.close();
-        }
+        boolean outputPartial = false;
+        return new PreclusteredGroupWriter(ctx, groupFields, groupByComparators,
+                aggregatorFactory, inputRecordDesc, outRecordDesc, nextWriter, outputPartial);
     }
 
-    private void merge(IFrameWriter mergeResultWriter, IFrameReader[] runCursors) throws HyracksDataException {
-        RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, mergeSortFields,
-                comparators, firstKeyNkc, partialAggRecordDesc);
-        merger.open();
-        try {
-            while (merger.nextFrame(outFrame)) {
-                FrameUtils.flushFrame(outFrame, mergeResultWriter);
-            }
-        } finally {
-            merger.close();
-        }
+    @Override
+    protected RunFileWriter prepareIntermediateMergeRunFile() throws HyracksDataException {
+        FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortGroupByRunMerger.class.getSimpleName());
+        return new RunFileWriter(newRun, ctx.getIOManager());
+    }
+
+    @Override
+    protected IFrameWriter prepareIntermediateMergeResultWriter(RunFileWriter mergeFileWriter)
+            throws HyracksDataException {
+        IAggregatorDescriptorFactory aggregatorFactory = localSide ? mergeAggregatorFactory : partialAggregatorFactory;
+        boolean outputPartial = true;
+        return new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators, aggregatorFactory,
+                partialAggRecordDesc, partialAggRecordDesc, mergeFileWriter, outputPartial);
+    }
+
+    @Override
+    protected IFrameWriter prepareFinalMergeResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
+        boolean outputPartial = false;
+        return new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators,
+                mergeAggregatorFactory, partialAggRecordDesc, outRecordDesc, nextWriter, outputPartial);
+    }
+
+    @Override
+    protected int[] getSortFields() {
+        return mergeSortFields;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
index cee105b..95cfb09 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
@@ -14,18 +14,11 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.group.sort;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
-import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.TaskId;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
@@ -34,34 +27,24 @@ import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.api.job.JobId;
-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.AbstractStateObject;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSorterOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
-import edu.uci.ics.hyracks.dataflow.std.sort.IFrameSorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
+import edu.uci.ics.hyracks.dataflow.std.sort.ISorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
 
 /**
  * This Operator pushes group-by aggregation into the external sort.
  * After the in-memory sort, it aggregates the sorted data before writing it to a run file.
  * During the merge phase, it does an aggregation over sorted results.
- * 
+ *
  * @author yingyib
  */
-public class SortGroupByOperatorDescriptor extends AbstractOperatorDescriptor {
-    private static final long serialVersionUID = 1L;
-
-    private static final int SORT_ACTIVITY_ID = 0;
-    private static final int MERGE_ACTIVITY_ID = 1;
+public class SortGroupByOperatorDescriptor extends AbstractSorterOperatorDescriptor {
 
-    private final int framesLimit;
-    private final int[] sortFields;
     private final int[] groupFields;
-    private final INormalizedKeyComputerFactory firstKeyNormalizerFactory;
-    private final IBinaryComparatorFactory[] comparatorFactories;
     private final IAggregatorDescriptorFactory mergeAggregatorFactory;
     private final IAggregatorDescriptorFactory partialAggregatorFactory;
     private final RecordDescriptor partialAggRecordDesc;
@@ -69,46 +52,31 @@ public class SortGroupByOperatorDescriptor extends AbstractOperatorDescriptor {
     private final boolean finalStage;
     private Algorithm alg = Algorithm.MERGE_SORT;
 
-    /***
-     * @param spec
-     *            , the Hyracks job specification
-     * @param framesLimit
-     *            , the frame limit for this operator
-     * @param sortFields
-     *            , the fields to sort
-     * @param groupFields
-     *            , the fields to group, which can be a prefix subset of sortFields
-     * @param firstKeyNormalizerFactory
-     *            , the normalized key computer factory of the first key
-     * @param comparatorFactories
-     *            , the comparator factories of sort keys
-     * @param partialAggregatorFactory
-     *            , for aggregating the input of this operator
-     * @param mergeAggregatorFactory
-     *            , for aggregating the intermediate data of this operator
-     * @param partialAggRecordDesc
-     *            , the record descriptor of intermediate data
-     * @param outRecordDesc
-     *            , the record descriptor of output data
-     * @param finalStage
-     *            , whether the operator is used for final stage aggregation
+    /**
+     * @param spec                      , the Hyracks job specification
+     * @param framesLimit               , the frame limit for this operator
+     * @param sortFields                , the fields to sort
+     * @param groupFields               , the fields to group, which can be a prefix subset of sortFields
+     * @param firstKeyNormalizerFactory , the normalized key computer factory of the first key
+     * @param comparatorFactories       , the comparator factories of sort keys
+     * @param partialAggregatorFactory  , for aggregating the input of this operator
+     * @param mergeAggregatorFactory    , for aggregating the intermediate data of this operator
+     * @param partialAggRecordDesc      , the record descriptor of intermediate data
+     * @param outRecordDesc             , the record descriptor of output data
+     * @param finalStage                , whether the operator is used for final stage aggregation
      */
     public SortGroupByOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
             int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
             IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory partialAggregatorFactory,
             IAggregatorDescriptorFactory mergeAggregatorFactory, RecordDescriptor partialAggRecordDesc,
             RecordDescriptor outRecordDesc, boolean finalStage) {
-        super(spec, 1, 1);
-        this.framesLimit = framesLimit;
-        this.sortFields = sortFields;
+
+        super(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, outRecordDesc);
         if (framesLimit <= 1) {
             throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
         }
-        this.recordDescriptors[0] = outRecordDesc;
 
         this.groupFields = groupFields;
-        this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
-        this.comparatorFactories = comparatorFactories;
         this.mergeAggregatorFactory = mergeAggregatorFactory;
         this.partialAggregatorFactory = partialAggregatorFactory;
         this.partialAggRecordDesc = partialAggRecordDesc;
@@ -116,31 +84,19 @@ public class SortGroupByOperatorDescriptor extends AbstractOperatorDescriptor {
         this.finalStage = finalStage;
     }
 
-    /***
-     * @param spec
-     *            , the Hyracks job specification
-     * @param framesLimit
-     *            , the frame limit for this operator
-     * @param sortFields
-     *            , the fields to sort
-     * @param groupFields
-     *            , the fields to group, which can be a prefix subset of sortFields
-     * @param firstKeyNormalizerFactory
-     *            , the normalized key computer factory of the first key
-     * @param comparatorFactories
-     *            , the comparator factories of sort keys
-     * @param partialAggregatorFactory
-     *            , for aggregating the input of this operator
-     * @param mergeAggregatorFactory
-     *            , for aggregating the intermediate data of this operator
-     * @param partialAggRecordDesc
-     *            , the record descriptor of intermediate data
-     * @param outRecordDesc
-     *            , the record descriptor of output data
-     * @param finalStage
-     *            , whether the operator is used for final stage aggregation
-     * @param alg
-     *            , the in-memory sort algorithm
+    /**
+     * @param spec                      , the Hyracks job specification
+     * @param framesLimit               , the frame limit for this operator
+     * @param sortFields                , the fields to sort
+     * @param groupFields               , the fields to group, which can be a prefix subset of sortFields
+     * @param firstKeyNormalizerFactory , the normalized key computer factory of the first key
+     * @param comparatorFactories       , the comparator factories of sort keys
+     * @param partialAggregatorFactory  , for aggregating the input of this operator
+     * @param mergeAggregatorFactory    , for aggregating the intermediate data of this operator
+     * @param partialAggRecordDesc      , the record descriptor of intermediate data
+     * @param outRecordDesc             , the record descriptor of output data
+     * @param finalStage                , whether the operator is used for final stage aggregation
+     * @param alg                       , the in-memory sort algorithm
      */
     public SortGroupByOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
             int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
@@ -153,122 +109,33 @@ public class SortGroupByOperatorDescriptor extends AbstractOperatorDescriptor {
     }
 
     @Override
-    public void contributeActivities(IActivityGraphBuilder builder) {
-        SortActivity sa = new SortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
-        MergeActivity ma = new MergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
-
-        builder.addActivity(this, sa);
-        builder.addSourceEdge(0, sa, 0);
-
-        builder.addActivity(this, ma);
-        builder.addTargetEdge(0, ma, 0);
-
-        builder.addBlockingEdge(sa, ma);
-    }
-
-    public static class SortTaskState extends AbstractStateObject {
-        private List<IFrameReader> runs;
-        private IFrameSorter frameSorter;
-
-        public SortTaskState() {
-        }
-
-        private SortTaskState(JobId jobId, TaskId taskId) {
-            super(jobId, taskId);
-        }
-
-        @Override
-        public void toBytes(DataOutput out) throws IOException {
-
-        }
-
-        @Override
-        public void fromBytes(DataInput in) throws IOException {
-
-        }
+    public AbstractSorterOperatorDescriptor.SortActivity getSortActivity(ActivityId id) {
+        return new AbstractSorterOperatorDescriptor.SortActivity(id) {
+            @Override
+            protected AbstractSortRunGenerator getRunGenerator(IHyracksTaskContext ctx,
+                    IRecordDescriptorProvider recordDescriptorProvider) throws HyracksDataException {
+                return new ExternalSortGroupByRunGenerator(ctx, sortFields,
+                        recordDescriptorProvider.getInputRecordDescriptor(this.getActivityId(), 0), framesLimit,
+                        groupFields, firstKeyNormalizerFactory, comparatorFactories, partialAggregatorFactory,
+                        partialAggRecordDesc, alg);
+            }
+        };
     }
 
-    private class SortActivity extends AbstractActivityNode {
-        private static final long serialVersionUID = 1L;
-
-        public SortActivity(ActivityId id) {
-            super(id);
-        }
-
-        @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
-            IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
-                private ExternalSortGroupByRunGenerator runGen;
-
-                @Override
-                public void open() throws HyracksDataException {
-                    runGen = new ExternalSortGroupByRunGenerator(ctx, sortFields,
-                            recordDescProvider.getInputRecordDescriptor(SortActivity.this.getActivityId(), 0),
-                            framesLimit, groupFields, firstKeyNormalizerFactory, comparatorFactories,
-                            partialAggregatorFactory, partialAggRecordDesc, alg);
-                    runGen.open();
-                }
-
-                @Override
-                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    runGen.nextFrame(buffer);
-                }
-
-                @Override
-                public void close() throws HyracksDataException {
-                    SortTaskState state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(
-                            getActivityId(), partition));
-                    runGen.close();
-                    state.runs = runGen.getRuns();
-                    state.frameSorter = runGen.getFrameSorter();
-                    ctx.setStateObject(state);
-                }
-
-                @Override
-                public void fail() throws HyracksDataException {
-                    runGen.fail();
-                }
-            };
-            return op;
-        }
-    }
-
-    private class MergeActivity extends AbstractActivityNode {
-        private static final long serialVersionUID = 1L;
-
-        public MergeActivity(ActivityId id) {
-            super(id);
-        }
-
-        @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
-            IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
-                @Override
-                public void initialize() throws HyracksDataException {
-                    SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
-                            SORT_ACTIVITY_ID), partition));
-                    List<IFrameReader> runs = state.runs;
-                    IFrameSorter frameSorter = state.frameSorter;
-                    int necessaryFrames = Math.min(runs.size() + 2, framesLimit);
-
-                    IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
-                    for (int i = 0; i < comparators.length; i++) {
-                        comparators[i] = comparatorFactories[i].createBinaryComparator();
-                    }
-                    INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory
-                            .createNormalizedKeyComputer();
-
-                    ExternalSortGroupByRunMerger merger = new ExternalSortGroupByRunMerger(ctx, frameSorter, runs,
-                            sortFields, recordDescProvider.getInputRecordDescriptor(new ActivityId(odId,
-                                    SORT_ACTIVITY_ID), 0), partialAggRecordDesc, outputRecordDesc, necessaryFrames,
-                            writer, groupFields, nkc, comparators, partialAggregatorFactory, mergeAggregatorFactory,
-                            !finalStage);
-                    merger.process();
-                }
-            };
-            return op;
-        }
+    @Override
+    public AbstractSorterOperatorDescriptor.MergeActivity getMergeActivity(ActivityId id) {
+        return new AbstractSorterOperatorDescriptor.MergeActivity(id) {
+
+            @Override
+            protected ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
+                    IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter,
+                    List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators,
+                    INormalizedKeyComputer nmkComputer, int necessaryFrames) {
+                return new ExternalSortGroupByRunMerger(ctx, sorter, runs, sortFields,
+                        recordDescProvider.getInputRecordDescriptor(new ActivityId(odId, SORT_ACTIVITY_ID), 0),
+                        partialAggRecordDesc, outputRecordDesc, necessaryFrames, writer, groupFields, nmkComputer,
+                        comparators, partialAggregatorFactory, mergeAggregatorFactory, !finalStage);
+            }
+        };
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
index 8375521..9c7fcf4 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
@@ -16,6 +16,8 @@ package edu.uci.ics.hyracks.dataflow.std.join;
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -103,7 +105,7 @@ class GraceHashJoinOperatorNodePushable extends AbstractUnaryOutputSourceOperato
 
         try {
 
-            ByteBuffer buffer = ctx.allocateFrame();// input
+            IFrame buffer = new VSizeFrame(ctx);
             // buffer
             int tableSize = (int) (numPartitions * recordsPerFrame * factor);
             ISerializableTable table = new SerializableHashTable(tableSize, ctx);
@@ -115,19 +117,19 @@ class GraceHashJoinOperatorNodePushable extends AbstractUnaryOutputSourceOperato
                     continue;
                 }
                 table.reset();
-                InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(
-                        ctx.getFrameSize(), rd0), hpcRep0, new FrameTupleAccessor(ctx.getFrameSize(), rd1), hpcRep1,
-                        new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter, nullWriters1, table, predEvaluator);
+                InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(rd0), hpcRep0,
+                        new FrameTupleAccessor(rd1), hpcRep1, new FrameTuplePairComparator(keys0, keys1, comparators),
+                        isLeftOuter, nullWriters1, table, predEvaluator);
 
                 // build
                 if (buildWriter != null) {
                     RunFileReader buildReader = buildWriter.createReader();
                     buildReader.open();
                     while (buildReader.nextFrame(buffer)) {
-                        ByteBuffer copyBuffer = ctx.allocateFrame();
-                        FrameUtils.copy(buffer, copyBuffer);
+                        ByteBuffer copyBuffer = ctx.allocateFrame(buffer.getFrameSize());
+                        FrameUtils.copyAndFlip(buffer.getBuffer(), copyBuffer);
                         joiner.build(copyBuffer);
-                        buffer.clear();
+                        buffer.reset();
                     }
                     buildReader.close();
                 }
@@ -136,8 +138,8 @@ class GraceHashJoinOperatorNodePushable extends AbstractUnaryOutputSourceOperato
                 RunFileReader probeReader = probeWriter.createReader();
                 probeReader.open();
                 while (probeReader.nextFrame(buffer)) {
-                    joiner.join(buffer, writer);
-                    buffer.clear();
+                    joiner.join(buffer.getBuffer(), writer);
+                    buffer.reset();
                 }
                 probeReader.close();
                 joiner.closeJoin(writer);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
index c6dbe61..70f28da 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
@@ -16,6 +16,8 @@ package edu.uci.ics.hyracks.dataflow.std.join;
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -39,7 +41,7 @@ class GraceHashJoinPartitionBuildOperatorNodePushable extends
     private final FrameTupleAccessor accessor0;
     private final ITuplePartitionComputer hpc;
     private final FrameTupleAppender appender;
-    private ByteBuffer[] outbufs;
+    private IFrame[] outbufs;
     private GraceHashJoinPartitionState state;
 
     GraceHashJoinPartitionBuildOperatorNodePushable(IHyracksTaskContext ctx, Object stateId, int[] keys,
@@ -48,8 +50,8 @@ class GraceHashJoinPartitionBuildOperatorNodePushable extends
         this.ctx = ctx;
         this.stateId = stateId;
         this.numPartitions = numPartitions;
-        accessor0 = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
-        appender = new FrameTupleAppender(ctx.getFrameSize());
+        accessor0 = new FrameTupleAccessor(inRecordDescriptor);
+        appender = new FrameTupleAppender();
         hpc = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories).createPartitioner();
         comparators = new IBinaryComparator[comparatorFactories.length];
         for (int i = 0; i < comparatorFactories.length; ++i) {
@@ -60,7 +62,7 @@ class GraceHashJoinPartitionBuildOperatorNodePushable extends
     @Override
     public void close() throws HyracksDataException {
         for (int i = 0; i < numPartitions; i++) {
-            ByteBuffer head = outbufs[i];
+            ByteBuffer head = outbufs[i].getBuffer();
             accessor0.reset(head);
             if (accessor0.getTupleCount() > 0) {
                 write(i, head);
@@ -97,13 +99,13 @@ class GraceHashJoinPartitionBuildOperatorNodePushable extends
         for (int i = 0; i < tCount; ++i) {
 
             int entry = hpc.partition(accessor0, i, numPartitions);
-            ByteBuffer outbuf = outbufs[entry];
+            IFrame outbuf = outbufs[entry];
             appender.reset(outbuf, false);
             if (!appender.append(accessor0, i)) {
                 // buffer is full, ie. we cannot fit the tuple
                 // into the buffer -- write it to disk
-                write(entry, outbuf);
-                outbuf.clear();
+                write(entry, outbuf.getBuffer());
+                outbuf.reset();
                 appender.reset(outbuf, true);
                 if (!appender.append(accessor0, i)) {
                     throw new HyracksDataException("Item too big to fit in frame");
@@ -115,10 +117,10 @@ class GraceHashJoinPartitionBuildOperatorNodePushable extends
     @Override
     public void open() throws HyracksDataException {
         state = new GraceHashJoinPartitionState(ctx.getJobletContext().getJobId(), stateId);
-        outbufs = new ByteBuffer[numPartitions];
+        outbufs = new IFrame[numPartitions];
         state.setRunWriters(new RunFileWriter[numPartitions]);
         for (int i = 0; i < numPartitions; i++) {
-            outbufs[i] = ctx.allocateFrame();
+            outbufs[i] = new VSizeFrame(ctx);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index 910edc7..0915ff9 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -19,6 +19,8 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
@@ -74,10 +76,8 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
 
     /**
      * @param spec
-     * @param memsize
-     *            in frames
-     * @param inputsize0
-     *            in frames
+     * @param memsize               in frames
+     * @param inputsize0            in frames
      * @param recordsPerFrame
      * @param factor
      * @param keys0
@@ -201,21 +201,21 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
                 private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(ctx.getJobletContext()
                         .getJobId(), new TaskId(getActivityId(), partition));
-                private final FrameTupleAccessor accessorBuild = new FrameTupleAccessor(ctx.getFrameSize(), rd1);
+                private final FrameTupleAccessor accessorBuild = new FrameTupleAccessor(rd1);
                 private final ITuplePartitionComputer hpcBuild = new FieldHashPartitionComputerFactory(keys1,
                         hashFunctionFactories).createPartitioner();
-                private final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-                private final FrameTupleAppender ftappender = new FrameTupleAppender(ctx.getFrameSize());
-                private ByteBuffer[] bufferForPartitions;
-                private final ByteBuffer inBuffer = ctx.allocateFrame();
+                private final FrameTupleAppender appender = new FrameTupleAppender();
+                private final FrameTupleAppender ftappender = new FrameTupleAppender();
+                private IFrame[] bufferForPartitions;
+                private final IFrame inBuffer = new VSizeFrame(ctx);
 
                 @Override
                 public void close() throws HyracksDataException {
                     if (state.memoryForHashtable != 0)
-                        build(inBuffer);
+                        build(inBuffer.getBuffer());
 
                     for (int i = 0; i < state.nPartitions; i++) {
-                        ByteBuffer buf = bufferForPartitions[i];
+                        ByteBuffer buf = bufferForPartitions[i].getBuffer();
                         accessorBuild.reset(buf);
                         if (accessorBuild.getTupleCount() > 0) {
                             write(i, buf);
@@ -233,18 +233,18 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                         accessorBuild.reset(buffer);
                         int tCount = accessorBuild.getTupleCount();
                         for (int i = 0; i < tCount; ++i) {
-                            int entry = -1;
+                            int entry;
                             if (state.memoryForHashtable == 0) {
                                 entry = hpcBuild.partition(accessorBuild, i, state.nPartitions);
                                 boolean newBuffer = false;
-                                ByteBuffer bufBi = bufferForPartitions[entry];
+                                IFrame bufBi = bufferForPartitions[entry];
                                 while (true) {
                                     appender.reset(bufBi, newBuffer);
                                     if (appender.append(accessorBuild, i)) {
                                         break;
                                     } else {
-                                        write(entry, bufBi);
-                                        bufBi.clear();
+                                        write(entry, bufBi.getBuffer());
+                                        bufBi.reset();
                                         newBuffer = true;
                                     }
                                 }
@@ -253,7 +253,7 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                                 if (entry < state.memoryForHashtable) {
                                     while (true) {
                                         if (!ftappender.append(accessorBuild, i)) {
-                                            build(inBuffer);
+                                            build(inBuffer.getBuffer());
 
                                             ftappender.reset(inBuffer, true);
                                         } else {
@@ -263,14 +263,14 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                                 } else {
                                     entry %= state.nPartitions;
                                     boolean newBuffer = false;
-                                    ByteBuffer bufBi = bufferForPartitions[entry];
+                                    IFrame bufBi = bufferForPartitions[entry];
                                     while (true) {
                                         appender.reset(bufBi, newBuffer);
                                         if (appender.append(accessorBuild, i)) {
                                             break;
                                         } else {
-                                            write(entry, bufBi);
-                                            bufBi.clear();
+                                            write(entry, bufBi.getBuffer());
+                                            bufBi.reset();
                                             newBuffer = true;
                                         }
                                     }
@@ -285,8 +285,8 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                 }
 
                 private void build(ByteBuffer inBuffer) throws HyracksDataException {
-                    ByteBuffer copyBuffer = ctx.allocateFrame();
-                    FrameUtils.copy(inBuffer, copyBuffer);
+                    ByteBuffer copyBuffer = ctx.allocateFrame(inBuffer.capacity());
+                    FrameUtils.copyAndFlip(inBuffer, copyBuffer);
                     state.joiner.build(copyBuffer);
                 }
 
@@ -321,13 +321,13 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                     int tableSize = (int) (state.memoryForHashtable * recordsPerFrame * factor);
                     ISerializableTable table = new SerializableHashTable(tableSize, ctx);
                     state.joiner = new InMemoryHashJoin(ctx, tableSize,
-                            new FrameTupleAccessor(ctx.getFrameSize(), rd0), hpc0, new FrameTupleAccessor(
-                                    ctx.getFrameSize(), rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
-                                    comparators), isLeftOuter, nullWriters1, table, predEvaluator);
-                    bufferForPartitions = new ByteBuffer[state.nPartitions];
+                            new FrameTupleAccessor(rd0), hpc0, new FrameTupleAccessor(rd1), hpc1,
+                            new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter, nullWriters1, table,
+                            predEvaluator);
+                    bufferForPartitions = new IFrame[state.nPartitions];
                     state.fWriters = new RunFileWriter[state.nPartitions];
                     for (int i = 0; i < state.nPartitions; i++) {
-                        bufferForPartitions[i] = ctx.allocateFrame();
+                        bufferForPartitions[i] = new VSizeFrame(ctx);
                     }
 
                     ftappender.reset(inBuffer, true);
@@ -391,20 +391,20 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
 
             IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
                 private BuildAndPartitionTaskState state;
-                private final FrameTupleAccessor accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), rd0);
+                private final FrameTupleAccessor accessorProbe = new FrameTupleAccessor(rd0);
                 private final ITuplePartitionComputerFactory hpcf0 = new FieldHashPartitionComputerFactory(keys0,
                         hashFunctionFactories);
                 private final ITuplePartitionComputerFactory hpcf1 = new FieldHashPartitionComputerFactory(keys1,
                         hashFunctionFactories);
                 private final ITuplePartitionComputer hpcProbe = hpcf0.createPartitioner();
 
-                private final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-                private final FrameTupleAppender ftap = new FrameTupleAppender(ctx.getFrameSize());
-                private final ByteBuffer inBuffer = ctx.allocateFrame();
-                private final ByteBuffer outBuffer = ctx.allocateFrame();
+                private final FrameTupleAppender appender = new FrameTupleAppender();
+                private final FrameTupleAppender ftap = new FrameTupleAppender();
+                private final IFrame inBuffer = new VSizeFrame(ctx);
+                private final IFrame outBuffer = new VSizeFrame(ctx);
                 private RunFileWriter[] buildWriters;
                 private RunFileWriter[] probeWriters;
-                private ByteBuffer[] bufferForPartitions;
+                private IFrame[] bufferForPartitions;
 
                 @Override
                 public void open() throws HyracksDataException {
@@ -413,9 +413,9 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                     writer.open();
                     buildWriters = state.fWriters;
                     probeWriters = new RunFileWriter[state.nPartitions];
-                    bufferForPartitions = new ByteBuffer[state.nPartitions];
+                    bufferForPartitions = new IFrame[state.nPartitions];
                     for (int i = 0; i < state.nPartitions; i++) {
-                        bufferForPartitions[i] = ctx.allocateFrame();
+                        bufferForPartitions[i] = new VSizeFrame(ctx);
                     }
                     appender.reset(outBuffer, true);
                     ftap.reset(inBuffer, true);
@@ -428,18 +428,18 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                         int tupleCount0 = accessorProbe.getTupleCount();
                         for (int i = 0; i < tupleCount0; ++i) {
 
-                            int entry = -1;
+                            int entry ;
                             if (state.memoryForHashtable == 0) {
                                 entry = hpcProbe.partition(accessorProbe, i, state.nPartitions);
                                 boolean newBuffer = false;
-                                ByteBuffer outbuf = bufferForPartitions[entry];
+                                IFrame outbuf = bufferForPartitions[entry];
                                 while (true) {
                                     appender.reset(outbuf, newBuffer);
                                     if (appender.append(accessorProbe, i)) {
                                         break;
                                     } else {
-                                        write(entry, outbuf);
-                                        outbuf.clear();
+                                        write(entry, outbuf.getBuffer());
+                                        outbuf.reset();
                                         newBuffer = true;
                                     }
                                 }
@@ -448,7 +448,7 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                                 if (entry < state.memoryForHashtable) {
                                     while (true) {
                                         if (!ftap.append(accessorProbe, i)) {
-                                            state.joiner.join(inBuffer, writer);
+                                            state.joiner.join(inBuffer.getBuffer(), writer);
                                             ftap.reset(inBuffer, true);
                                         } else
                                             break;
@@ -457,14 +457,14 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                                 } else {
                                     entry %= state.nPartitions;
                                     boolean newBuffer = false;
-                                    ByteBuffer outbuf = bufferForPartitions[entry];
+                                    IFrame outbuf = bufferForPartitions[entry];
                                     while (true) {
                                         appender.reset(outbuf, newBuffer);
                                         if (appender.append(accessorProbe, i)) {
                                             break;
                                         } else {
-                                            write(entry, outbuf);
-                                            outbuf.clear();
+                                            write(entry, outbuf.getBuffer());
+                                            outbuf.reset();
                                             newBuffer = true;
                                         }
                                     }
@@ -478,7 +478,7 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
 
                 @Override
                 public void close() throws HyracksDataException {
-                    state.joiner.join(inBuffer, writer);
+                    state.joiner.join(inBuffer.getBuffer(), writer);
                     state.joiner.closeJoin(writer);
                     ITuplePartitionComputer hpcRep0 = new RepartitionComputerFactory(state.nPartitions, hpcf0)
                             .createPartitioner();
@@ -486,7 +486,7 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                             .createPartitioner();
                     if (state.memoryForHashtable != memsize - 2) {
                         for (int i = 0; i < state.nPartitions; i++) {
-                            ByteBuffer buf = bufferForPartitions[i];
+                            ByteBuffer buf = bufferForPartitions[i].getBuffer();
                             accessorProbe.reset(buf);
                             if (accessorProbe.getTupleCount() > 0) {
                                 write(i, buf);
@@ -494,7 +494,7 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                             closeWriter(i);
                         }
 
-                        inBuffer.clear();
+                        inBuffer.reset();
                         int tableSize = -1;
                         if (state.memoryForHashtable == 0) {
                             tableSize = (int) (state.nPartitions * recordsPerFrame * factor);
@@ -510,18 +510,18 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                             }
                             table.reset();
                             InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(
-                                    ctx.getFrameSize(), rd0), hpcRep0, new FrameTupleAccessor(ctx.getFrameSize(), rd1),
-                                    hpcRep1, new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter,
-                                    nullWriters1, table, predEvaluator);
+                                    rd0), hpcRep0, new FrameTupleAccessor(rd1), hpcRep1,
+                                    new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter, nullWriters1,
+                                    table, predEvaluator);
 
                             if (buildWriter != null) {
                                 RunFileReader buildReader = buildWriter.createReader();
                                 buildReader.open();
                                 while (buildReader.nextFrame(inBuffer)) {
-                                    ByteBuffer copyBuffer = ctx.allocateFrame();
-                                    FrameUtils.copy(inBuffer, copyBuffer);
+                                    ByteBuffer copyBuffer = ctx.allocateFrame(inBuffer.getFrameSize());
+                                    FrameUtils.copyAndFlip(inBuffer.getBuffer(), copyBuffer);
                                     joiner.build(copyBuffer);
-                                    inBuffer.clear();
+                                    inBuffer.reset();
                                 }
                                 buildReader.close();
                             }
@@ -530,8 +530,8 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                             RunFileReader probeReader = probeWriter.createReader();
                             probeReader.open();
                             while (probeReader.nextFrame(inBuffer)) {
-                                joiner.join(inBuffer, writer);
-                                inBuffer.clear();
+                                joiner.join(inBuffer.getBuffer(), writer);
+                                inBuffer.reset();
                             }
                             probeReader.close();
                             joiner.closeJoin(writer);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
index 860cdd4..d6a83d2 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -21,6 +21,7 @@ import java.util.List;
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
@@ -30,6 +31,7 @@ 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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.structures.ISerializableTable;
 import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
 
@@ -43,7 +45,6 @@ public class InMemoryHashJoin {
     private final ITuplePartitionComputer tpcProbe;
     private final FrameTupleAppender appender;
     private final FrameTuplePairComparator tpComparator;
-    private final ByteBuffer outBuffer;
     private final boolean isLeftOuter;
     private final ArrayTupleBuilder nullTupleBuild;
     private final ISerializableTable table;
@@ -75,10 +76,8 @@ public class InMemoryHashJoin {
         this.tpcBuild = tpc1;
         this.accessorProbe = accessor0;
         this.tpcProbe = tpc0;
-        appender = new FrameTupleAppender(ctx.getFrameSize());
+        appender = new FrameTupleAppender(new VSizeFrame(ctx));
         tpComparator = comparator;
-        outBuffer = ctx.allocateFrame();
-        appender.reset(outBuffer, true);
         predEvaluator = predEval;
         this.isLeftOuter = isLeftOuter;
         if (isLeftOuter) {
@@ -136,24 +135,15 @@ public class InMemoryHashJoin {
                 } while (true);
             }
             if (!matchFound && isLeftOuter) {
-                if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
-                        nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
-                    flushFrame(outBuffer, writer);
-                    appender.reset(outBuffer, true);
-                    if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
-                            nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
-                        throw new HyracksDataException("Record size larger than frame size ("
-                                + appender.getBuffer().capacity() + ")");
-                    }
-                }
+                FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, i,
+                        nullTupleBuild.getFieldEndOffsets(), nullTupleBuild.getByteArray(), 0,
+                        nullTupleBuild.getSize());
             }
         }
     }
 
     public void closeJoin(IFrameWriter writer) throws HyracksDataException {
-        if (appender.getTupleCount() > 0) {
-            flushFrame(outBuffer, writer);
-        }
+        appender.flush(writer, true);
         int nFrames = buffers.size();
         buffers.clear();
         ctx.deallocateFrames(nFrames);
@@ -179,31 +169,11 @@ public class InMemoryHashJoin {
 
     private void appendToResult(int probeSidetIx, int buildSidetIx, IFrameWriter writer) throws HyracksDataException {
         if (!reverseOutputOrder) {
-            if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
-                flushFrame(outBuffer, writer);
-                appender.reset(outBuffer, true);
-                if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
-                    int tSize = accessorProbe.getTupleEndOffset(probeSidetIx)
-                            - accessorProbe.getTupleStartOffset(probeSidetIx)
-                            + accessorBuild.getTupleEndOffset(buildSidetIx)
-                            - accessorBuild.getTupleStartOffset(buildSidetIx);
-                    throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
-                            + appender.getBuffer().capacity() + ")");
-                }
-            }
+            FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, probeSidetIx, accessorBuild,
+                    buildSidetIx);
         } else {
-            if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
-                flushFrame(outBuffer, writer);
-                appender.reset(outBuffer, true);
-                if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
-                    int tSize = accessorProbe.getTupleEndOffset(probeSidetIx)
-                            - accessorProbe.getTupleStartOffset(probeSidetIx)
-                            + accessorBuild.getTupleEndOffset(buildSidetIx)
-                            - accessorBuild.getTupleStartOffset(buildSidetIx);
-                    throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
-                            + appender.getBuffer().capacity() + ")");
-                }
-            }
+            FrameUtils.appendConcatToWriter(writer, appender, accessorBuild, buildSidetIx, accessorProbe,
+                    probeSidetIx);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index be369d2..35e22ad 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -77,7 +77,8 @@ public class InMemoryHashJoinOperatorDescriptor extends AbstractOperatorDescript
 
     public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
-            IPredicateEvaluatorFactory predEvalFactory, RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1,
+            IPredicateEvaluatorFactory predEvalFactory, RecordDescriptor recordDescriptor, boolean isLeftOuter,
+            INullWriterFactory[] nullWriterFactories1,
             int tableSize) {
         super(spec, 2, 1);
         this.keys0 = keys0;
@@ -90,21 +91,20 @@ public class InMemoryHashJoinOperatorDescriptor extends AbstractOperatorDescript
         this.nullWriterFactories1 = nullWriterFactories1;
         this.tableSize = tableSize;
     }
-    
+
     public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor, int tableSize) {
-    	this(spec, keys0, keys1, hashFunctionFactories, comparatorFactories, recordDescriptor, tableSize, null);
+        this(spec, keys0, keys1, hashFunctionFactories, comparatorFactories, recordDescriptor, tableSize, null);
     }
-    
+
     public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
             IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1,
             int tableSize) {
-    	this(spec, keys0, keys1, hashFunctionFactories, comparatorFactories,null,recordDescriptor,isLeftOuter,nullWriterFactories1,tableSize);
+        this(spec, keys0, keys1, hashFunctionFactories, comparatorFactories, null, recordDescriptor, isLeftOuter,
+                nullWriterFactories1, tableSize);
     }
-    
-    
 
     @Override
     public void contributeActivities(IActivityGraphBuilder builder) {
@@ -170,7 +170,9 @@ public class InMemoryHashJoinOperatorDescriptor extends AbstractOperatorDescript
                     nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
                 }
             }
-            final IPredicateEvaluator predEvaluator = ( predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator());
+            final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ?
+                    null :
+                    predEvaluatorFactory.createPredicateEvaluator());
 
             IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
                 private HashBuildTaskState state;
@@ -185,15 +187,15 @@ public class InMemoryHashJoinOperatorDescriptor extends AbstractOperatorDescript
                             partition));
                     ISerializableTable table = new SerializableHashTable(tableSize, ctx);
                     state.joiner = new InMemoryHashJoin(ctx, tableSize,
-                            new FrameTupleAccessor(ctx.getFrameSize(), rd0), hpc0, new FrameTupleAccessor(
-                                    ctx.getFrameSize(), rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
+                            new FrameTupleAccessor(rd0), hpc0, new FrameTupleAccessor(rd1), hpc1,
+                            new FrameTuplePairComparator(keys0, keys1,
                                     comparators), isLeftOuter, nullWriters1, table, predEvaluator);
                 }
 
                 @Override
                 public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    ByteBuffer copyBuffer = ctx.allocateFrame();
-                    FrameUtils.copy(buffer, copyBuffer);
+                    ByteBuffer copyBuffer = ctx.allocateFrame(buffer.capacity());
+                    FrameUtils.copyAndFlip(buffer, copyBuffer);
                     state.joiner.build(copyBuffer);
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
index eab60bc..3bd0540 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -19,7 +19,9 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
 import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
@@ -38,8 +40,8 @@ public class NestedLoopJoin {
     private final FrameTupleAccessor accessorOuter;
     private final FrameTupleAppender appender;
     private final ITuplePairComparator tpComparator;
-    private final ByteBuffer outBuffer;
-    private final ByteBuffer innerBuffer;
+    private final IFrame outBuffer;
+    private final IFrame innerBuffer;
     private final List<ByteBuffer> outBuffers;
     private final int memSize;
     private final IHyracksTaskContext ctx;
@@ -49,18 +51,18 @@ public class NestedLoopJoin {
     private final boolean isLeftOuter;
     private final ArrayTupleBuilder nullTupleBuilder;
     private final IPredicateEvaluator predEvaluator;
-    private boolean isReversed;		//Added for handling correct calling for predicate-evaluator upon recursive calls (in OptimizedHybridHashJoin) that cause role-reversal
+    private boolean isReversed;        //Added for handling correct calling for predicate-evaluator upon recursive calls (in OptimizedHybridHashJoin) that cause role-reversal
 
-    
     public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
-            ITuplePairComparator comparators, int memSize, IPredicateEvaluator predEval, boolean isLeftOuter, INullWriter[] nullWriters1)
+            ITuplePairComparator comparators, int memSize, IPredicateEvaluator predEval, boolean isLeftOuter,
+            INullWriter[] nullWriters1)
             throws HyracksDataException {
         this.accessorInner = accessor1;
         this.accessorOuter = accessor0;
-        this.appender = new FrameTupleAppender(ctx.getFrameSize());
+        this.appender = new FrameTupleAppender();
         this.tpComparator = comparators;
-        this.outBuffer = ctx.allocateFrame();
-        this.innerBuffer = ctx.allocateFrame();
+        this.outBuffer = new VSizeFrame(ctx);
+        this.innerBuffer = new VSizeFrame(ctx);
         this.appender.reset(outBuffer, true);
         this.outBuffers = new ArrayList<ByteBuffer>();
         this.memSize = memSize;
@@ -107,7 +109,7 @@ public class NestedLoopJoin {
         runFileReader.open();
         while (runFileReader.nextFrame(innerBuffer)) {
             for (ByteBuffer outBuffer : outBuffers) {
-                blockJoin(outBuffer, innerBuffer, writer);
+                blockJoin(outBuffer, innerBuffer.getBuffer(), writer);
             }
         }
         runFileReader.close();
@@ -116,15 +118,18 @@ public class NestedLoopJoin {
     }
 
     private void createAndCopyFrame(ByteBuffer outerBuffer) throws HyracksDataException {
-        ByteBuffer outerBufferCopy = ctx.allocateFrame();
-        FrameUtils.copy(outerBuffer, outerBufferCopy);
+        ByteBuffer outerBufferCopy = ctx.allocateFrame(outerBuffer.capacity());
+        FrameUtils.copyAndFlip(outerBuffer, outerBufferCopy);
         outBuffers.add(outerBufferCopy);
         currentMemSize++;
     }
 
-    private void reloadFrame(ByteBuffer outerBuffer) {
+    private void reloadFrame(ByteBuffer outerBuffer) throws HyracksDataException {
         outBuffers.get(currentMemSize).clear();
-        FrameUtils.copy(outerBuffer, outBuffers.get(currentMemSize));
+        if (outBuffers.get(currentMemSize).capacity() != outerBuffer.capacity()) {
+            outBuffers.set(currentMemSize, ctx.allocateFrame(outerBuffer.capacity()));
+        }
+        FrameUtils.copyAndFlip(outerBuffer, outBuffers.get(currentMemSize));
         currentMemSize++;
     }
 
@@ -141,8 +146,8 @@ public class NestedLoopJoin {
                 int c = compare(accessorOuter, i, accessorInner, j);
                 boolean prdEval = evaluatePredicate(i, j);
                 if (c == 0 && prdEval) {
-                	matchFound = true;
-                	appendToResults(i, j, writer);
+                    matchFound = true;
+                    appendToResults(i, j, writer);
                 }
             }
 
@@ -150,28 +155,20 @@ public class NestedLoopJoin {
                 final int[] ntFieldEndOffsets = nullTupleBuilder.getFieldEndOffsets();
                 final byte[] ntByteArray = nullTupleBuilder.getByteArray();
                 final int ntSize = nullTupleBuilder.getSize();
-                if (!appender.appendConcat(accessorOuter, i, ntFieldEndOffsets, ntByteArray, 0, ntSize)) {
-                    flushFrame(outBuffer, writer);
-                    appender.reset(outBuffer, true);
-                    if (!appender.appendConcat(accessorOuter, i, ntFieldEndOffsets, ntByteArray, 0, ntSize)) {
-                        int tSize = accessorOuter.getTupleEndOffset(i) - accessorOuter.getTupleStartOffset(i) + ntSize;
-                        throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
-                                + appender.getBuffer().capacity() + ")");
-                    }
-                }
+                FrameUtils.appendConcatToWriter(writer, appender, accessorOuter, i, ntFieldEndOffsets, ntByteArray, 0,
+                        ntSize);
             }
         }
     }
-    
-    private boolean evaluatePredicate(int tIx1, int tIx2){
-    	if(isReversed){		//Role Reversal Optimization is triggered
-    		return ( (predEvaluator == null) || predEvaluator.evaluate(accessorInner, tIx2, accessorOuter, tIx1) );
-    	}
-    	else {
-    		return ( (predEvaluator == null) || predEvaluator.evaluate(accessorOuter, tIx1, accessorInner, tIx2) );
-    	}
+
+    private boolean evaluatePredicate(int tIx1, int tIx2) {
+        if (isReversed) {        //Role Reversal Optimization is triggered
+            return ((predEvaluator == null) || predEvaluator.evaluate(accessorInner, tIx2, accessorOuter, tIx1));
+        } else {
+            return ((predEvaluator == null) || predEvaluator.evaluate(accessorOuter, tIx1, accessorInner, tIx2));
+        }
     }
-    
+
     private void appendToResults(int outerTupleId, int innerTupleId, IFrameWriter writer) throws HyracksDataException {
         if (!isReversed) {
             appendResultToFrame(accessorOuter, outerTupleId, accessorInner, innerTupleId, writer);
@@ -183,18 +180,9 @@ public class NestedLoopJoin {
 
     private void appendResultToFrame(FrameTupleAccessor accessor1, int tupleId1, FrameTupleAccessor accessor2,
             int tupleId2, IFrameWriter writer) throws HyracksDataException {
-        if (!appender.appendConcat(accessor1, tupleId1, accessor2, tupleId2)) {
-            flushFrame(outBuffer, writer);
-            appender.reset(outBuffer, true);
-            if (!appender.appendConcat(accessor1, tupleId1, accessor2, tupleId2)) {
-                int tSize = accessor1.getTupleEndOffset(tupleId1) - accessor1.getTupleStartOffset(tupleId1)
-                        + accessor2.getTupleEndOffset(tupleId2) - accessor2.getTupleStartOffset(tupleId2);
-                throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
-                        + appender.getBuffer().capacity() + ")");
-            }
-        }
+        FrameUtils.appendConcatToWriter(writer, appender, accessor1, tupleId1, accessor2, tupleId2);
     }
-    
+
     public void closeCache() throws HyracksDataException {
         if (runFileWriter != null) {
             runFileWriter.close();
@@ -206,24 +194,14 @@ public class NestedLoopJoin {
         runFileReader.open();
         while (runFileReader.nextFrame(innerBuffer)) {
             for (int i = 0; i < currentMemSize; i++) {
-                blockJoin(outBuffers.get(i), innerBuffer, writer);
+                blockJoin(outBuffers.get(i), innerBuffer.getBuffer(), writer);
             }
         }
         runFileReader.close();
         outBuffers.clear();
         currentMemSize = 0;
 
-        if (appender.getTupleCount() > 0) {
-            flushFrame(outBuffer, writer);
-        }
-    }
-
-    private void flushFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
-        writer.nextFrame(buffer);
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
+        appender.flush(writer, true);
     }
 
     private int compare(FrameTupleAccessor accessor0, int tIndex0, FrameTupleAccessor accessor1, int tIndex1)
@@ -234,8 +212,8 @@ public class NestedLoopJoin {
         }
         return 0;
     }
-    
-    public void setIsReversed(boolean b){
-    	this.isReversed = b;
+
+    public void setIsReversed(boolean b) {
+        this.isReversed = b;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index 41f0f4f..3e06bf3 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -54,16 +54,17 @@ public class NestedLoopJoinOperatorDescriptor extends AbstractOperatorDescriptor
     private final IPredicateEvaluatorFactory predEvaluatorFactory;
     private final boolean isLeftOuter;
     private final INullWriterFactory[] nullWriterFactories1;
-    
+
     public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
             ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
             boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
         this(spec, comparatorFactory, recordDescriptor, memSize, null, isLeftOuter, nullWriterFactories1);
     }
-    
+
     public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
             ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
-            IPredicateEvaluatorFactory predEvalFactory, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
+            IPredicateEvaluatorFactory predEvalFactory, boolean isLeftOuter,
+            INullWriterFactory[] nullWriterFactories1) {
         super(spec, 2, 1);
         this.comparatorFactory = comparatorFactory;
         this.recordDescriptors[0] = recordDescriptor;
@@ -127,8 +128,10 @@ public class NestedLoopJoinOperatorDescriptor extends AbstractOperatorDescriptor
             final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
             final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
             final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator(ctx);
-            final IPredicateEvaluator predEvaluator = ( (predEvaluatorFactory != null) ? predEvaluatorFactory.createPredicateEvaluator() : null);
-            
+            final IPredicateEvaluator predEvaluator = ((predEvaluatorFactory != null) ?
+                    predEvaluatorFactory.createPredicateEvaluator() :
+                    null);
+
             final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
             if (isLeftOuter) {
                 for (int i = 0; i < nullWriterFactories1.length; i++) {
@@ -144,17 +147,16 @@ public class NestedLoopJoinOperatorDescriptor extends AbstractOperatorDescriptor
                     state = new JoinCacheTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(),
                             partition));
 
-                    state.joiner = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), rd0),
-                            new FrameTupleAccessor(ctx.getFrameSize(), rd1), comparator, memSize, predEvaluator, isLeftOuter,
+                    state.joiner = new NestedLoopJoin(ctx, new FrameTupleAccessor(rd0),
+                            new FrameTupleAccessor(rd1), comparator, memSize, predEvaluator, isLeftOuter,
                             nullWriters1);
 
                 }
 
                 @Override
                 public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    ByteBuffer copyBuffer = ctx.allocateFrame();
-                    FrameUtils.copy(buffer, copyBuffer);
-                    FrameUtils.makeReadable(copyBuffer);
+                    ByteBuffer copyBuffer = ctx.allocateFrame(buffer.capacity());
+                    FrameUtils.copyAndFlip(buffer, copyBuffer);
                     state.joiner.cache(copyBuffer);
                 }
 


[07/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index 9178094..6b36480 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -1,318 +1,271 @@
 /*
  * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
+ *  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.
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
  */
 package edu.uci.ics.hyracks.dataflow.std.sort;
 
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
 import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
-
-/**
- * @author pouria This class defines the logic for merging the run, generated
- *         during the first phase of external sort (for both sorting without
- *         replacement selection and with it). For the case with replacement
- *         selection, this code also takes the limit on the output into account
- *         (if specified). If number of input runs is less than the available
- *         memory frames, then merging can be done in one pass, by allocating
- *         one buffer per run, and one buffer as the output buffer. A
- *         priorityQueue is used to find the top tuple at each iteration, among
- *         all the runs' heads in memory (check RunMergingFrameReader for more
- *         details). Otherwise, assuming that we have R runs and M memory
- *         buffers, where (R > M), we first merge first (M-1) runs and create a
- *         new sorted run, out of them. Discarding the first (M-1) runs, now
- *         merging procedure gets applied recursively on the (R-M+2) remaining
- *         runs using the M memory buffers. For the case of replacement
- *         selection, if outputLimit is specified, once the final pass is done
- *         on the runs (which is the pass that generates the final sorted
- *         output), as soon as the output size hits the output limit, the
- *         process stops, closes, and returns.
- */
+import edu.uci.ics.hyracks.dataflow.std.sort.util.GroupVSizeFrame;
 
 public class ExternalSortRunMerger {
 
-    private final IHyracksTaskContext ctx;
-    private final List<IFrameReader> runs;
+    protected final IHyracksTaskContext ctx;
+    protected final IFrameWriter writer;
+
+    private final List<RunAndMaxFrameSizePair> runs;
+    private final BitSet currentGenerationRunAvailable;
     private final int[] sortFields;
     private final IBinaryComparator[] comparators;
     private final INormalizedKeyComputer nmkComputer;
     private final RecordDescriptor recordDesc;
     private final int framesLimit;
-    private final IFrameWriter writer;
-    private List<ByteBuffer> inFrames;
-    private ByteBuffer outFrame;
-    private FrameTupleAppender outFrameAppender;
-
-    private IFrameSorter frameSorter; // Used in External sort, no replacement
-                                      // selection
-    private FrameTupleAccessor outFrameAccessor; // Used in External sort, with
-                                                 // replacement selection
-    private final int outputLimit; // Used in External sort, with replacement
-                                   // selection and limit on output size
-    private int currentSize; // Used in External sort, with replacement
-                             // selection and limit on output size
-
-    // Constructor for external sort, no replacement selection
-    public ExternalSortRunMerger(IHyracksTaskContext ctx, IFrameSorter frameSorter, List<IFrameReader> runs,
+    private final int MAX_FRAME_SIZE;
+    private final ArrayList<IFrameReader> tempRuns;
+    private final int topK;
+    private List<GroupVSizeFrame> inFrames;
+    private VSizeFrame outputFrame;
+    private ISorter sorter;
+
+    private static final Logger LOGGER = Logger.getLogger(ExternalSortRunMerger.class.getName());
+
+    public ExternalSortRunMerger(IHyracksTaskContext ctx, ISorter sorter, List<RunAndMaxFrameSizePair> runs,
             int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
             RecordDescriptor recordDesc, int framesLimit, IFrameWriter writer) {
-        this.ctx = ctx;
-        this.frameSorter = frameSorter;
-        this.runs = new LinkedList<IFrameReader>(runs);
-        this.sortFields = sortFields;
-        this.comparators = comparators;
-        this.nmkComputer = nmkComputer;
-        this.recordDesc = recordDesc;
-        this.framesLimit = framesLimit;
-        this.writer = writer;
-        this.outputLimit = -1;
+        this(ctx, sorter, runs, sortFields, comparators, nmkComputer, recordDesc, framesLimit,
+                Integer.MAX_VALUE, writer);
     }
 
-    // Constructor for external sort with replacement selection
-    public ExternalSortRunMerger(IHyracksTaskContext ctx, int outputLimit, List<IFrameReader> runs, int[] sortFields,
-            IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc,
-            int framesLimit, IFrameWriter writer) {
+    public ExternalSortRunMerger(IHyracksTaskContext ctx, ISorter sorter, List<RunAndMaxFrameSizePair> runs,
+            int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+            RecordDescriptor recordDesc, int framesLimit, int topK, IFrameWriter writer) {
         this.ctx = ctx;
-        this.runs = new LinkedList<IFrameReader>(runs);
+        this.sorter = sorter;
+        this.runs = new LinkedList<>(runs);
+        this.currentGenerationRunAvailable = new BitSet(runs.size());
         this.sortFields = sortFields;
         this.comparators = comparators;
         this.nmkComputer = nmkComputer;
         this.recordDesc = recordDesc;
         this.framesLimit = framesLimit;
         this.writer = writer;
-        this.outputLimit = outputLimit;
-        this.currentSize = 0;
-        this.frameSorter = null;
+        this.MAX_FRAME_SIZE = FrameConstants.MAX_NUM_MINFRAME * ctx.getInitialFrameSize();
+        this.topK = topK;
+        this.tempRuns = new ArrayList<>(runs.size());
     }
 
     public void process() throws HyracksDataException {
-        writer.open();
+        IFrameWriter finalWriter = null;
         try {
             if (runs.size() <= 0) {
-                if (frameSorter != null && frameSorter.getFrameCount() > 0) {
-                    frameSorter.flushFrames(writer);
+                finalWriter = prepareSkipMergingFinalResultWriter(writer);
+                finalWriter.open();
+                if (sorter != null) {
+                    if (sorter.hasRemaining()) {
+                        sorter.flush(finalWriter);
+                    }
+                    sorter.close();
                 }
-                /** recycle sort buffer */
-                frameSorter.close();
             } else {
                 /** recycle sort buffer */
-                frameSorter.close();
-
-                inFrames = new ArrayList<ByteBuffer>();
-                outFrame = ctx.allocateFrame();
-                outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
-                outFrameAppender.reset(outFrame, true);
-                for (int i = 0; i < framesLimit - 1; ++i) {
-                    inFrames.add(ctx.allocateFrame());
+                if (sorter != null) {
+                    sorter.close();
                 }
+
+                finalWriter = prepareFinalMergeResultWriter(writer);
+                finalWriter.open();
+
                 int maxMergeWidth = framesLimit - 1;
-                while (runs.size() > maxMergeWidth) {
-                    int generationSeparator = 0;
-                    while (generationSeparator < runs.size() && runs.size() > maxMergeWidth) {
-                        int mergeWidth = Math.min(Math.min(runs.size() - generationSeparator, maxMergeWidth),
-                                runs.size() - maxMergeWidth + 1);
-                        FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.class
-                                .getSimpleName());
-                        IFrameWriter mergeResultWriter = new RunFileWriter(newRun, ctx.getIOManager());
-                        mergeResultWriter.open();
-                        IFrameReader[] runCursors = new RunFileReader[mergeWidth];
-                        for (int i = 0; i < mergeWidth; i++) {
-                            runCursors[i] = runs.get(generationSeparator + i);
+
+                inFrames = new ArrayList<>(maxMergeWidth);
+                outputFrame = new VSizeFrame(ctx);
+                List<RunAndMaxFrameSizePair> partialRuns = new ArrayList<>(maxMergeWidth);
+
+                int stop = runs.size();
+                currentGenerationRunAvailable.set(0, stop);
+
+                while (true) {
+
+                    int unUsed = selectPartialRuns(maxMergeWidth * ctx.getInitialFrameSize(), runs, partialRuns,
+                            currentGenerationRunAvailable,
+                            stop);
+                    prepareFrames(unUsed, inFrames, partialRuns);
+
+                    if (!currentGenerationRunAvailable.isEmpty() || stop < runs.size()) {
+                        IFrameReader reader;
+                        int mergedMaxFrameSize;
+                        if (partialRuns.size() == 1) {
+                            if (!currentGenerationRunAvailable.isEmpty()) {
+                                throw new HyracksDataException(
+                                        "The record is too big to put into the merging frame, please"
+                                                + " allocate more sorting memory");
+                            } else {
+                                reader = partialRuns.get(0).run;
+                                mergedMaxFrameSize = partialRuns.get(0).maxFrameSize;
+                            }
+
+                        } else {
+                            RunFileWriter mergeFileWriter = prepareIntermediateMergeRunFile();
+                            IFrameWriter mergeResultWriter = prepareIntermediateMergeResultWriter(mergeFileWriter);
+
+                            mergeResultWriter.open();
+                            mergedMaxFrameSize = merge(mergeResultWriter, partialRuns);
+                            mergeResultWriter.close();
+
+                            reader = mergeFileWriter.createReader();
                         }
-                        merge(mergeResultWriter, runCursors);
-                        mergeResultWriter.close();
-                        runs.subList(generationSeparator, mergeWidth + generationSeparator).clear();
-                        runs.add(generationSeparator++, ((RunFileWriter) mergeResultWriter).createReader());
-                    }
-                }
-                if (!runs.isEmpty()) {
-                    IFrameReader[] runCursors = new RunFileReader[runs.size()];
-                    for (int i = 0; i < runCursors.length; i++) {
-                        runCursors[i] = runs.get(i);
+
+                        appendNewRuns(reader, mergedMaxFrameSize);
+                        if (currentGenerationRunAvailable.isEmpty()) {
+
+                            if (LOGGER.isLoggable(Level.FINE)) {
+                                LOGGER.fine("generated runs:" + stop);
+                            }
+                            runs.subList(0, stop).clear();
+                            currentGenerationRunAvailable.clear();
+                            currentGenerationRunAvailable.set(0, runs.size());
+                            stop = runs.size();
+                        }
+                    } else {
+                        if (LOGGER.isLoggable(Level.FINE)) {
+                            LOGGER.fine("final runs:" + stop);
+                        }
+                        merge(finalWriter, partialRuns);
+                        break;
                     }
-                    merge(writer, runCursors);
                 }
             }
         } catch (Exception e) {
-            writer.fail();
+            finalWriter.fail();
             throw new HyracksDataException(e);
         } finally {
-            writer.close();
+            finalWriter.close();
         }
     }
 
-    private void merge(IFrameWriter mergeResultWriter, IFrameReader[] runCursors) throws HyracksDataException {
-        RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields, comparators,
-                nmkComputer, recordDesc);
-        merger.open();
-        try {
-            while (merger.nextFrame(outFrame)) {
-                FrameUtils.flushFrame(outFrame, mergeResultWriter);
+    private void appendNewRuns(IFrameReader reader, int mergedPartialMaxSize) {
+        runs.add(new RunAndMaxFrameSizePair(reader, mergedPartialMaxSize));
+    }
+
+    private static int selectPartialRuns(int budget, List<RunAndMaxFrameSizePair> runs,
+            List<RunAndMaxFrameSizePair> partialRuns, BitSet runAvailable, int stop) {
+        partialRuns.clear();
+        int maxFrameSizeOfGenRun = 0;
+        int nextRunId = runAvailable.nextSetBit(0);
+        while (budget > 0 && nextRunId >= 0 && nextRunId < stop) {
+            int runFrameSize = runs.get(nextRunId).maxFrameSize;
+            if (budget - runFrameSize >= 0) {
+                partialRuns.add(runs.get(nextRunId));
+                budget -= runFrameSize;
+                runAvailable.clear(nextRunId);
+                maxFrameSizeOfGenRun = runFrameSize > maxFrameSizeOfGenRun ? runFrameSize : maxFrameSizeOfGenRun;
             }
-        } finally {
-            merger.close();
+            nextRunId = runAvailable.nextSetBit(nextRunId + 1);
         }
+        return budget;
     }
 
-    public void processWithReplacementSelection() throws HyracksDataException {
-        writer.open();
-        try {
-            outFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
-            outFrame = ctx.allocateFrame();
-            outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
-            outFrameAppender.reset(outFrame, true);
-            if (runs.size() == 1) {
-                if (outputLimit < 1) {
-                    runs.get(0).open();
-                    ByteBuffer nextFrame = ctx.allocateFrame();
-                    while (runs.get(0).nextFrame(nextFrame)) {
-                        FrameUtils.flushFrame(nextFrame, writer);
-                        outFrameAppender.reset(nextFrame, true);
-                    }
-                    return;
-                }
-                // Limit on the output size
-                int totalCount = 0;
-                runs.get(0).open();
-                FrameTupleAccessor fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
-                ByteBuffer nextFrame = ctx.allocateFrame();
-                while (totalCount <= outputLimit && runs.get(0).nextFrame(nextFrame)) {
-                    fta.reset(nextFrame);
-                    int tupCount = fta.getTupleCount();
-                    if ((totalCount + tupCount) < outputLimit) {
-                        FrameUtils.flushFrame(nextFrame, writer);
-                        totalCount += tupCount;
-                        continue;
-                    }
-                    // The very last buffer, which exceeds the limit
-                    int copyCount = outputLimit - totalCount;
-                    outFrameAppender.reset(outFrame, true);
-                    for (int i = 0; i < copyCount; i++) {
-                        if (!outFrameAppender.append(fta, i)) {
-                            throw new HyracksDataException("Record size ("
-                                    + (fta.getTupleEndOffset(i) - fta.getTupleStartOffset(i))
-                                    + ") larger than frame size (" + outFrameAppender.getBuffer().capacity() + ")");
-                        }
-                        totalCount++;
-                    }
-                }
-                if (outFrameAppender.getTupleCount() > 0) {
-                    FrameUtils.flushFrame(outFrame, writer);
-                    outFrameAppender.reset(outFrame, true);
-                }
-                return;
+    private void prepareFrames(int extraFreeMem, List<GroupVSizeFrame> inFrames,
+            List<RunAndMaxFrameSizePair> patialRuns)
+            throws HyracksDataException {
+        if (extraFreeMem > 0 && patialRuns.size() > 1) {
+            int extraFrames = extraFreeMem / ctx.getInitialFrameSize();
+            int avg = (extraFrames / patialRuns.size()) * ctx.getInitialFrameSize();
+            int residue = (extraFrames % patialRuns.size());
+            for (int i = 0; i < residue; i++) {
+                patialRuns.get(i).updateSize(
+                        Math.min(MAX_FRAME_SIZE, patialRuns.get(i).maxFrameSize + avg + ctx.getInitialFrameSize()));
             }
-            // More than one run, actual merging is needed
-            inFrames = new ArrayList<ByteBuffer>();
-            for (int i = 0; i < framesLimit - 1; ++i) {
-                inFrames.add(ctx.allocateFrame());
-            }
-            while (runs.size() > 0) {
-                try {
-                    doPassWithReplacementSelection(runs);
-                } catch (Exception e) {
-                    throw new HyracksDataException(e);
-                }
+            for (int i = residue; i < patialRuns.size() && avg > 0; i++) {
+                patialRuns.get(i).updateSize(Math.min(MAX_FRAME_SIZE, patialRuns.get(i).maxFrameSize + avg));
             }
+        }
 
-        } catch (Exception e) {
-            writer.fail();
-            throw new HyracksDataException(e);
-        } finally {
-            writer.close();
+        if (inFrames.size() > patialRuns.size()) {
+            inFrames.subList(patialRuns.size(), inFrames.size()).clear();
+        }
+        int i;
+        for (i = 0; i < inFrames.size(); i++) {
+            inFrames.get(i).resize(patialRuns.get(i).maxFrameSize);
+        }
+        for (; i < patialRuns.size(); i++) {
+            inFrames.add(new GroupVSizeFrame(ctx, patialRuns.get(i).maxFrameSize));
         }
     }
 
-    // creates a new run from runs that can fit in memory.
-    private void doPassWithReplacementSelection(List<IFrameReader> runs) throws HyracksDataException {
-        FileReference newRun = null;
-        IFrameWriter writer = this.writer;
-        boolean finalPass = false;
-        if (runs.size() + 1 <= framesLimit) { // + 1 outFrame
-            finalPass = true;
-            for (int i = inFrames.size() - 1; i >= runs.size(); i--) {
-                inFrames.remove(i);
-            }
-        } else {
-            newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
-            writer = new RunFileWriter(newRun, ctx.getIOManager());
-            writer.open();
-        }
-        try {
-            IFrameReader[] runCursors = new RunFileReader[inFrames.size()];
-            for (int i = 0; i < inFrames.size(); i++) {
-                runCursors[i] = runs.get(i);
-            }
-            RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields,
-                    comparators, nmkComputer, recordDesc);
-            merger.open();
-            try {
-                while (merger.nextFrame(outFrame)) {
-                    if (outputLimit > 0 && finalPass) {
-                        outFrameAccessor.reset(outFrame);
-                        int count = outFrameAccessor.getTupleCount();
-                        if ((currentSize + count) > outputLimit) {
-                            ByteBuffer b = ctx.allocateFrame();
-                            FrameTupleAppender partialAppender = new FrameTupleAppender(ctx.getFrameSize());
-                            partialAppender.reset(b, true);
-                            int copyCount = outputLimit - currentSize;
-                            for (int i = 0; i < copyCount; i++) {
-                                partialAppender.append(outFrameAccessor, i);
-                                currentSize++;
-                            }
-                            FrameUtils.makeReadable(b);
-                            FrameUtils.flushFrame(b, writer);
-                            break;
-                        } else {
-                            FrameUtils.flushFrame(outFrame, writer);
-                            currentSize += count;
-                        }
-                    } else {
-                        FrameUtils.flushFrame(outFrame, writer);
-                    }
-                }
-            } finally {
-                merger.close();
-            }
+    protected IFrameWriter prepareSkipMergingFinalResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
+        return nextWriter;
+    }
 
-            if (outputLimit > 0 && finalPass && (currentSize >= outputLimit)) {
-                runs.clear();
-                return;
-            }
+    protected RunFileWriter prepareIntermediateMergeRunFile() throws HyracksDataException {
+        FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
+        return new RunFileWriter(newRun, ctx.getIOManager());
+    }
+
+    protected IFrameWriter prepareIntermediateMergeResultWriter(RunFileWriter mergeFileWriter)
+            throws HyracksDataException {
+        return mergeFileWriter;
+    }
+
+    protected IFrameWriter prepareFinalMergeResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
+        return nextWriter;
+    }
+
+    protected int[] getSortFields() {
+        return sortFields;
+    }
 
-            runs.subList(0, inFrames.size()).clear();
-            if (!finalPass) {
-                runs.add(0, ((RunFileWriter) writer).createReader());
+    private int merge(IFrameWriter writer, List<RunAndMaxFrameSizePair> partialRuns)
+            throws HyracksDataException {
+        tempRuns.clear();
+        for (int i = 0; i < partialRuns.size(); i++) {
+            tempRuns.add(partialRuns.get(i).run);
+        }
+        RunMergingFrameReader merger = new RunMergingFrameReader(ctx, tempRuns, inFrames, getSortFields(),
+                comparators, nmkComputer, recordDesc, topK);
+        int maxFrameSize = 0;
+        int io = 0;
+        merger.open();
+        try {
+            while (merger.nextFrame(outputFrame)) {
+                FrameUtils.flushFrame(outputFrame.getBuffer(), writer);
+                maxFrameSize = maxFrameSize < outputFrame.getFrameSize() ? outputFrame.getFrameSize() : maxFrameSize;
+                io++;
             }
         } finally {
-            if (!finalPass) {
-                writer.close();
+            merger.close();
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Output " + io + " frames");
             }
         }
+        return maxFrameSize;
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
index 8dbdbd4..82a8453 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
@@ -1,161 +1,69 @@
 /*
  * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
+ *  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.
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
  */
 package edu.uci.ics.hyracks.dataflow.std.sort;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
 
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
 
-public class FrameSorterMergeSort implements IFrameSorter {
-    private final IHyracksTaskContext ctx;
-    private final int[] sortFields;
-    private final INormalizedKeyComputer nkc;
-    private final IBinaryComparator[] comparators;
-    private final List<ByteBuffer> buffers;
+public class FrameSorterMergeSort extends AbstractFrameSorter {
 
-    private final FrameTupleAccessor fta1;
-    private final FrameTupleAccessor fta2;
-
-    private final FrameTupleAppender appender;
-
-    private final ByteBuffer outFrame;
-
-    private int dataFrameCount;
-    private int[] tPointers;
     private int[] tPointersTemp;
-    private int tupleCount;
+    private FrameTupleAccessor fta2;
 
-    public FrameSorterMergeSort(IHyracksTaskContext ctx, int[] sortFields,
+    public FrameSorterMergeSort(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor) throws HyracksDataException {
-        this.ctx = ctx;
-        this.sortFields = sortFields;
-        nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
-        comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
-        buffers = new ArrayList<ByteBuffer>();
-        fta1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        outFrame = ctx.allocateFrame();
-
-        dataFrameCount = 0;
+        this(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+                Integer.MAX_VALUE);
     }
 
-    @Override
-    public void reset() {
-        dataFrameCount = 0;
-        tupleCount = 0;
-    }
-
-    @Override
-    public int getFrameCount() {
-        return dataFrameCount;
-    }
-
-    @Override
-    public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
-        ByteBuffer copyFrame;
-        if (dataFrameCount == buffers.size()) {
-            copyFrame = ctx.allocateFrame();
-            buffers.add(copyFrame);
-        } else {
-            copyFrame = buffers.get(dataFrameCount);
-        }
-        FrameUtils.copy(buffer, copyFrame);
-        ++dataFrameCount;
+    public FrameSorterMergeSort(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, int outputLimit) throws HyracksDataException {
+        super(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+                outputLimit);
+        fta2 = new FrameTupleAccessor(recordDescriptor);
     }
 
     @Override
-    public void sortFrames() throws HyracksDataException {
-        int nBuffers = dataFrameCount;
-        tupleCount = 0;
-        for (int i = 0; i < nBuffers; ++i) {
-            fta1.reset(buffers.get(i));
-            tupleCount += fta1.getTupleCount();
-        }
-        int sfIdx = sortFields[0];
-        tPointers = tPointers == null || tPointers.length < tupleCount * 4 ? new int[tupleCount * 4] : tPointers;
-        int ptr = 0;
-        for (int i = 0; i < nBuffers; ++i) {
-            fta1.reset(buffers.get(i));
-            int tCount = fta1.getTupleCount();
-            byte[] array = fta1.getBuffer().array();
-            for (int j = 0; j < tCount; ++j) {
-                int tStart = fta1.getTupleStartOffset(j);
-                int tEnd = fta1.getTupleEndOffset(j);
-                tPointers[ptr * 4] = i;
-                tPointers[ptr * 4 + 1] = tStart;
-                tPointers[ptr * 4 + 2] = tEnd;
-                int f0StartRel = fta1.getFieldStartOffset(j, sfIdx);
-                int f0EndRel = fta1.getFieldEndOffset(j, sfIdx);
-                int f0Start = f0StartRel + tStart + fta1.getFieldSlotsLength();
-                tPointers[ptr * 4 + 3] = nkc == null ? 0 : nkc.normalize(array, f0Start, f0EndRel - f0StartRel);
-                ++ptr;
-            }
-        }
-        if (tupleCount > 0) {
+    void sortTupleReferences() throws HyracksDataException {
+        if (tPointersTemp == null || tPointersTemp.length < tPointers.length) {
             tPointersTemp = new int[tPointers.length];
-            sort(0, tupleCount);
         }
+        sort(0, tupleCount);
     }
 
     @Override
-    public void flushFrames(IFrameWriter writer) throws HyracksDataException {
-        appender.reset(outFrame, true);
-        for (int ptr = 0; ptr < tupleCount; ++ptr) {
-            int i = tPointers[ptr * 4];
-            int tStart = tPointers[ptr * 4 + 1];
-            int tEnd = tPointers[ptr * 4 + 2];
-            ByteBuffer buffer = buffers.get(i);
-            fta1.reset(buffer);
-            if (!appender.append(fta1, tStart, tEnd)) {
-                FrameUtils.flushFrame(outFrame, writer);
-                appender.reset(outFrame, true);
-                if (!appender.append(fta1, tStart, tEnd)) {
-                    throw new HyracksDataException("Record size (" + (tEnd - tStart) + ") larger than frame size ("
-                            + appender.getBuffer().capacity() + ")");
-                }
-            }
-        }
-        if (appender.getTupleCount() > 0) {
-            FrameUtils.flushFrame(outFrame, writer);
-        }
+    public void close() {
+        super.close();
+        tPointersTemp = null;
     }
 
-    private void sort(int offset, int length) throws HyracksDataException {
+    void sort(int offset, int length) throws HyracksDataException {
         int step = 1;
-        int len = length;
-        int end = offset + len;
+        int end = offset + length;
         /** bottom-up merge */
-        while (step < len) {
+        while (step < length) {
             /** merge */
             for (int i = offset; i < end; i += 2 * step) {
                 int next = i + step;
@@ -175,8 +83,6 @@ public class FrameSorterMergeSort implements IFrameSorter {
 
     /**
      * Merge two subarrays into one
-     *
-     * @throws HyracksDataException
      */
     private void merge(int start1, int start2, int len1, int len2) throws HyracksDataException {
         int targetPos = start1;
@@ -226,20 +132,20 @@ public class FrameSorterMergeSort implements IFrameSorter {
         }
         int i2 = tp2i;
         int j2 = tp2j;
-        ByteBuffer buf1 = buffers.get(i1);
-        ByteBuffer buf2 = buffers.get(i2);
+        ByteBuffer buf1 = super.bufferManager.getFrame(i1);
+        ByteBuffer buf2 = super.bufferManager.getFrame(i2);
         byte[] b1 = buf1.array();
         byte[] b2 = buf2.array();
-        fta1.reset(buf1);
+        inputTupleAccessor.reset(buf1);
         fta2.reset(buf2);
         for (int f = 0; f < comparators.length; ++f) {
             int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf1.array(), j1 + (fIdx - 1) * 4);
-            int f1End = IntSerDeUtils.getInt(buf1.array(), j1 + fIdx * 4);
-            int s1 = j1 + fta1.getFieldSlotsLength() + f1Start;
+            int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(b1, j1 + (fIdx - 1) * 4);
+            int f1End = IntSerDeUtils.getInt(b1, j1 + fIdx * 4);
+            int s1 = j1 + inputTupleAccessor.getFieldSlotsLength() + f1Start;
             int l1 = f1End - f1Start;
-            int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf2.array(), j2 + (fIdx - 1) * 4);
-            int f2End = IntSerDeUtils.getInt(buf2.array(), j2 + fIdx * 4);
+            int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(b2, j2 + (fIdx - 1) * 4);
+            int f2End = IntSerDeUtils.getInt(b2, j2 + fIdx * 4);
             int s2 = j2 + fta2.getFieldSlotsLength() + f2Start;
             int l2 = f2End - f2Start;
             int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
@@ -250,8 +156,4 @@ public class FrameSorterMergeSort implements IFrameSorter {
         return 0;
     }
 
-    @Override
-    public void close() {
-        this.buffers.clear();
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
index d607a51..328bb5e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
@@ -1,153 +1,54 @@
 /*
  * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
+ *  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.
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
  */
 package edu.uci.ics.hyracks.dataflow.std.sort;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
 
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
 
-public class FrameSorterQuickSort implements IFrameSorter {
-    private final IHyracksTaskContext ctx;
-    private final int[] sortFields;
-    private final INormalizedKeyComputer nkc;
-    private final IBinaryComparator[] comparators;
-    private final List<ByteBuffer> buffers;
+public class FrameSorterQuickSort extends AbstractFrameSorter {
 
-    private final FrameTupleAccessor fta1;
-    private final FrameTupleAccessor fta2;
+    private FrameTupleAccessor fta2;
 
-    private final FrameTupleAppender appender;
-
-    private final ByteBuffer outFrame;
-
-    private int dataFrameCount;
-    private int[] tPointers;
-    private int tupleCount;
-
-    public FrameSorterQuickSort(IHyracksTaskContext ctx, int[] sortFields,
+    public FrameSorterQuickSort(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor) throws HyracksDataException {
-        this.ctx = ctx;
-        this.sortFields = sortFields;
-        nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
-        comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
-        buffers = new ArrayList<ByteBuffer>();
-        fta1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        outFrame = ctx.allocateFrame();
-
-        dataFrameCount = 0;
-    }
-
-    @Override
-    public void reset() {
-        dataFrameCount = 0;
-        tupleCount = 0;
-    }
-
-    @Override
-    public int getFrameCount() {
-        return dataFrameCount;
+        this(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+                Integer.MAX_VALUE);
     }
 
-    @Override
-    public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
-        ByteBuffer copyFrame;
-        if (dataFrameCount == buffers.size()) {
-            copyFrame = ctx.allocateFrame();
-            buffers.add(copyFrame);
-        } else {
-            copyFrame = buffers.get(dataFrameCount);
-        }
-        FrameUtils.copy(buffer, copyFrame);
-        ++dataFrameCount;
-    }
-
-    @Override
-    public void sortFrames() throws HyracksDataException {
-        int nBuffers = dataFrameCount;
-        tupleCount = 0;
-        for (int i = 0; i < nBuffers; ++i) {
-            fta1.reset(buffers.get(i));
-            tupleCount += fta1.getTupleCount();
-        }
-        int sfIdx = sortFields[0];
-        tPointers = tPointers == null || tPointers.length < tupleCount * 4 ? new int[tupleCount * 4] : tPointers;
-        int ptr = 0;
-        for (int i = 0; i < nBuffers; ++i) {
-            fta1.reset(buffers.get(i));
-            int tCount = fta1.getTupleCount();
-            byte[] array = fta1.getBuffer().array();
-            for (int j = 0; j < tCount; ++j) {
-                int tStart = fta1.getTupleStartOffset(j);
-                int tEnd = fta1.getTupleEndOffset(j);
-                tPointers[ptr * 4] = i;
-                tPointers[ptr * 4 + 1] = tStart;
-                tPointers[ptr * 4 + 2] = tEnd;
-                int f0StartRel = fta1.getFieldStartOffset(j, sfIdx);
-                int f0EndRel = fta1.getFieldEndOffset(j, sfIdx);
-                int f0Start = f0StartRel + tStart + fta1.getFieldSlotsLength();
-                tPointers[ptr * 4 + 3] = nkc == null ? 0 : nkc.normalize(array, f0Start, f0EndRel - f0StartRel);
-                ++ptr;
-            }
-        }
-        if (tupleCount > 0) {
-            sort(tPointers, 0, tupleCount);
-        }
+    public FrameSorterQuickSort(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, int outputLimit) throws HyracksDataException {
+        super(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+                outputLimit);
+        fta2 = new FrameTupleAccessor(recordDescriptor);
     }
 
     @Override
-    public void flushFrames(IFrameWriter writer) throws HyracksDataException {
-        appender.reset(outFrame, true);
-        for (int ptr = 0; ptr < tupleCount; ++ptr) {
-            int i = tPointers[ptr * 4];
-            int tStart = tPointers[ptr * 4 + 1];
-            int tEnd = tPointers[ptr * 4 + 2];
-            ByteBuffer buffer = buffers.get(i);
-            fta1.reset(buffer);
-            if (!appender.append(fta1, tStart, tEnd)) {
-                FrameUtils.flushFrame(outFrame, writer);
-                appender.reset(outFrame, true);
-                if (!appender.append(fta1, tStart, tEnd)) {
-                    throw new HyracksDataException("Record size (" + (tEnd - tStart) + ") larger than frame size ("
-                            + appender.getBuffer().capacity() + ")");
-                }
-            }
-        }
-        if (appender.getTupleCount() > 0) {
-            FrameUtils.flushFrame(outFrame, writer);
-        }
+    void sortTupleReferences() throws HyracksDataException {
+        sort(tPointers, 0, tupleCount);
     }
 
-    private void sort(int[] tPointers, int offset, int length) throws HyracksDataException {
+    void sort(int[] tPointers, int offset, int length) throws HyracksDataException {
         int m = offset + (length >> 1);
         int mi = tPointers[m * 4];
         int mj = tPointers[m * 4 + 1];
@@ -221,17 +122,17 @@ public class FrameSorterQuickSort implements IFrameSorter {
         }
         int i2 = tp2i;
         int j2 = tp2j;
-        ByteBuffer buf1 = buffers.get(i1);
-        ByteBuffer buf2 = buffers.get(i2);
+        ByteBuffer buf1 = super.bufferManager.getFrame(i1);
+        ByteBuffer buf2 = super.bufferManager.getFrame(i2);
         byte[] b1 = buf1.array();
         byte[] b2 = buf2.array();
-        fta1.reset(buf1);
+        inputTupleAccessor.reset(buf1);
         fta2.reset(buf2);
         for (int f = 0; f < comparators.length; ++f) {
             int fIdx = sortFields[f];
             int f1Start = fIdx == 0 ? 0 : buf1.getInt(j1 + (fIdx - 1) * 4);
             int f1End = buf1.getInt(j1 + fIdx * 4);
-            int s1 = j1 + fta1.getFieldSlotsLength() + f1Start;
+            int s1 = j1 + inputTupleAccessor.getFieldSlotsLength() + f1Start;
             int l1 = f1End - f1Start;
             int f2Start = fIdx == 0 ? 0 : buf2.getInt(j2 + (fIdx - 1) * 4);
             int f2End = buf2.getInt(j2 + fIdx * 4);
@@ -245,8 +146,4 @@ public class FrameSorterQuickSort implements IFrameSorter {
         return 0;
     }
 
-    @Override
-    public void close() {
-        this.buffers.clear();
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HeapSortRunGenerator.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HeapSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HeapSortRunGenerator.java
new file mode 100644
index 0000000..564a462
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HeapSortRunGenerator.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFramePool;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.ITupleBufferManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableTupleMemoryManager;
+
+public class HeapSortRunGenerator extends AbstractSortRunGenerator {
+    protected final IHyracksTaskContext ctx;
+    protected final int frameLimit;
+    protected final int topK;
+    protected final int[] sortFields;
+    protected final INormalizedKeyComputerFactory nmkFactory;
+    protected final IBinaryComparatorFactory[] comparatorFactories;
+    protected final RecordDescriptor recordDescriptor;
+    protected ITupleSorter tupleSorter;
+    protected IFrameTupleAccessor inAccessor;
+
+    public HeapSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int topK, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor) {
+        super();
+        this.ctx = ctx;
+        this.frameLimit = frameLimit;
+        this.topK = topK;
+        this.sortFields = sortFields;
+        this.nmkFactory = firstKeyNormalizerFactory;
+        this.comparatorFactories = comparatorFactories;
+        this.inAccessor = new FrameTupleAccessor(recordDescriptor);
+        this.recordDescriptor = recordDescriptor;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        IFramePool framePool = new VariableFramePool(ctx, (frameLimit - 1) * ctx.getInitialFrameSize());
+        ITupleBufferManager bufferManager = new VariableTupleMemoryManager(framePool, recordDescriptor);
+        tupleSorter = new TupleSorterHeapSort(ctx, bufferManager, topK, sortFields, nmkFactory,
+                comparatorFactories);
+        super.open();
+    }
+
+    @Override
+    public ISorter getSorter() throws HyracksDataException {
+        return tupleSorter;
+    }
+
+    @Override
+    protected RunFileWriter getRunFileWriter() throws HyracksDataException {
+        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+                HeapSortRunGenerator.class.getSimpleName());
+        return new RunFileWriter(file, ctx.getIOManager());
+    }
+
+    @Override
+    protected IFrameWriter getFlushableFrameWriter(RunFileWriter writer) throws HyracksDataException {
+        return writer;
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        inAccessor.reset(buffer);
+        for (int i = 0; i < inAccessor.getTupleCount(); i++) {
+            if (!tupleSorter.insertTuple(inAccessor, i)) {
+                flushFramesToRun();
+                if (!tupleSorter.insertTuple(inAccessor, i)) {
+                    throw new HyracksDataException("The given tuple is too big to insert into the sorting memory.");
+                }
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
new file mode 100644
index 0000000..9976aad
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotBiggestFirst;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
+
+public class HybridTopKSortRunGenerator extends HeapSortRunGenerator {
+    private static final Logger LOG = Logger.getLogger(HybridTopKSortRunGenerator.class.getName());
+
+    private static final int SWITCH_TO_FRAME_SORTER_THRESHOLD = 2;
+    private IFrameSorter frameSorter = null;
+    private int tupleSorterFlushedTimes = 0;
+
+    public HybridTopKSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int topK, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor) {
+        super(ctx, frameLimit, topK, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
+    }
+
+    @Override
+    public ISorter getSorter() throws HyracksDataException {
+        if (tupleSorter != null) {
+            return tupleSorter;
+        } else if (frameSorter != null) {
+            return frameSorter;
+        }
+        return null;
+    }
+
+    @Override
+    protected RunFileWriter getRunFileWriter() throws HyracksDataException {
+        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+                HybridTopKSortRunGenerator.class.getSimpleName());
+        return new RunFileWriter(file, ctx.getIOManager());
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        inAccessor.reset(buffer);
+        if (tupleSorter != null) {
+            boolean isBadK = false;
+            for (int i = 0; i < inAccessor.getTupleCount(); i++) {
+                if (!tupleSorter.insertTuple(inAccessor, i)) {
+                    flushFramesToRun();
+                    isBadK = true;
+                    if (!tupleSorter.insertTuple(inAccessor, i)) {
+                        throw new HyracksDataException("The given tuple is too big to insert into the sorting memory.");
+                    }
+                }
+            }
+            if (isBadK) {
+                tupleSorterFlushedTimes++;
+                if (tupleSorterFlushedTimes > SWITCH_TO_FRAME_SORTER_THRESHOLD) {
+                    if (tupleSorter.hasRemaining()) {
+                        flushFramesToRun();
+                    }
+                    tupleSorter.close();
+                    tupleSorter = null;
+                    if (LOG.isLoggable(Level.FINE)) {
+                        LOG.fine("clear tupleSorter");
+                    }
+                }
+            }
+        } else {
+            if (frameSorter == null) {
+                VariableFrameMemoryManager bufferManager = new VariableFrameMemoryManager(
+                        new VariableFramePool(ctx, (frameLimit - 1) * ctx.getInitialFrameSize()),
+                        new FrameFreeSlotBiggestFirst(frameLimit - 1));
+                frameSorter = new FrameSorterMergeSort(ctx, bufferManager, sortFields, nmkFactory, comparatorFactories,
+                        recordDescriptor, topK);
+                if (LOG.isLoggable(Level.FINE)) {
+                    LOG.fine("create frameSorter");
+                }
+            }
+            if (!frameSorter.insertFrame(buffer)) {
+                flushFramesToRun();
+                if (!frameSorter.insertFrame(buffer)) {
+                    throw new HyracksDataException("The given frame is too big to insert into the sorting memory.");
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
index d9b8d37..6d0b100 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
@@ -1,37 +1,28 @@
 /*
  * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
+ *  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.
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
  */
 
 package edu.uci.ics.hyracks.dataflow.std.sort;
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
-public interface IFrameSorter {
+public interface IFrameSorter extends ISorter {
 
-    public void reset();
+    int getFrameCount();
 
-    public int getFrameCount();
-
-    public void insertFrame(ByteBuffer buffer) throws HyracksDataException;
-
-    public void sortFrames() throws HyracksDataException;
-
-    public void flushFrames(IFrameWriter writer) throws HyracksDataException;
-
-    public void close();
+    boolean insertFrame(ByteBuffer inputBuffer) throws HyracksDataException;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IMemoryManager.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IMemoryManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IMemoryManager.java
deleted file mode 100644
index e669335..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IMemoryManager.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-
-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;
-
-/**
- * @author pouria Defines the required operations, needed for any memory
- *         manager, used in sorting with replacement selection, to manage the
- *         free spaces
- */
-
-public interface IMemoryManager {
-
-    /**
-     * Allocates a free slot equal or greater than requested length. Pointer to
-     * the allocated slot is put in result, and gets returned to the caller. If
-     * no proper free slot is available, result would contain a null/invalid
-     * pointer (may vary between different implementations)
-     * 
-     * @param length
-     * @param result
-     * @throws HyracksDataException
-     */
-    void allocate(int length, Slot result) throws HyracksDataException;
-
-    /**
-     * Unallocates the specified slot (and returns it back to the free slots
-     * set)
-     * 
-     * @param s
-     * @return the total length of unallocted slot
-     * @throws HyracksDataException
-     */
-    int unallocate(Slot s) throws HyracksDataException;
-
-    /**
-     * @param frameIndex
-     * @return the specified frame, from the set of memory buffers, being
-     *         managed by this memory manager
-     */
-    ByteBuffer getFrame(int frameIndex);
-
-    /**
-     * Writes the specified tuple into the specified memory slot (denoted by
-     * frameIx and offset)
-     * 
-     * @param frameIx
-     * @param offset
-     * @param src
-     * @param tIndex
-     * @return
-     */
-    boolean writeTuple(int frameIx, int offset, FrameTupleAccessor src, int tIndex);
-
-    /**
-     * Reads the specified tuple (denoted by frameIx and offset) and appends it
-     * to the passed FrameTupleAppender
-     * 
-     * @param frameIx
-     * @param offset
-     * @param dest
-     * @return
-     */
-    boolean readTuple(int frameIx, int offset, FrameTupleAppender dest);
-
-    /**
-     * close and cleanup the memory manager
-     */
-    void close();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IRunGenerator.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IRunGenerator.java
index 2840d01..d21255e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IRunGenerator.java
@@ -16,7 +16,6 @@ package edu.uci.ics.hyracks.dataflow.std.sort;
 
 import java.util.List;
 
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 
 /**
@@ -28,5 +27,5 @@ public interface IRunGenerator extends IFrameWriter {
     /**
      * @return the list of generated (sorted) runs
      */
-    public List<IFrameReader> getRuns();
+    List<RunAndMaxFrameSizePair> getRuns();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISelectionTree.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISelectionTree.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISelectionTree.java
deleted file mode 100644
index 8cff0df..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISelectionTree.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * @author pouria
- *         Defines the selection tree, used in sorting with replacement
- *         selection to manage the order of output tuples into the runs, during
- *         the run generation phase. This tree contains tuples, belonging to two
- *         different runs: - Current run (being written to the output) - Next
- *         run
- */
-
-public interface ISelectionTree {
-
-    /**
-     * Inserts a new element into the selectionTree
-     *
-     * @param element
-     *            contains the pointer to the memory slot, containing the tuple,
-     *            along with its run number
-     * @throws HyracksDataException
-     */
-    void insert(int[] element) throws HyracksDataException;
-
-    /**
-     * Removes and returns the smallest element in the tree
-     *
-     * @param result
-     *            is the array that will eventually contain minimum entry
-     *            pointer
-     * @throws HyracksDataException
-     */
-    void getMin(int[] result) throws HyracksDataException;
-
-    /**
-     * Removes and returns the largest element in the tree
-     *
-     * @param result
-     *            is the array that will eventually contain maximum entry
-     *            pointer
-     * @throws HyracksDataException
-     */
-    void getMax(int[] result) throws HyracksDataException;
-
-    /**
-     * @return True of the selection tree does not have any element, false
-     *         otherwise
-     */
-    boolean isEmpty();
-
-    /**
-     * Removes all the elements in the tree
-     */
-    void reset();
-
-    /**
-     * Returns (and does NOT remove) the smallest element in the tree
-     *
-     * @param result
-     *            is the array that will eventually contain minimum entry
-     *            pointer
-     */
-    void peekMin(int[] result);
-
-    /**
-     * Returns (and does NOT remove) the largest element in the tree
-     *
-     * @param result
-     *            is the array that will eventually contain maximum entry
-     *            pointer
-     * @throws HyracksDataException
-     */
-    void peekMax(int[] result) throws HyracksDataException;
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISorter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISorter.java
new file mode 100644
index 0000000..09a8169
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISorter.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ISorter {
+
+    boolean hasRemaining();
+
+    void reset() throws HyracksDataException;
+
+    void sort() throws HyracksDataException;
+
+    void close();
+
+    int flush(IFrameWriter writer) throws HyracksDataException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ITupleSorter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ITupleSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ITupleSorter.java
new file mode 100644
index 0000000..bea8b35
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ITupleSorter.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ITupleSorter extends ISorter {
+    int getTupleCount();
+
+    boolean insertTuple(IFrameTupleAccessor frameTupleAccessor, int index) throws HyracksDataException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index 6fa21b5..379a783 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -36,6 +36,10 @@ import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotLastFit;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
 
 public class InMemorySortOperatorDescriptor extends AbstractOperatorDescriptor {
     private static final long serialVersionUID = 1L;
@@ -76,7 +80,7 @@ public class InMemorySortOperatorDescriptor extends AbstractOperatorDescriptor {
         builder.addBlockingEdge(sa, ma);
     }
 
-    public static class SortTaskState extends AbstractStateObject {
+    private static class SortTaskState extends AbstractStateObject {
         private FrameSorterMergeSort frameSorter;
 
         public SortTaskState() {
@@ -110,20 +114,29 @@ public class InMemorySortOperatorDescriptor extends AbstractOperatorDescriptor {
 
                 @Override
                 public void open() throws HyracksDataException {
-                    state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition));
-                    state.frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory,
-                            comparatorFactories, recordDescriptors[0]);
+                    state = new SortTaskState(ctx.getJobletContext().getJobId(),
+                            new TaskId(getActivityId(), partition));
+
+                    IFrameBufferManager frameBufferManager = new VariableFrameMemoryManager(
+                            new VariableFramePool(ctx, VariableFramePool.UNLIMITED_MEMORY), new FrameFreeSlotLastFit());
+
+                    state.frameSorter = new FrameSorterMergeSort(ctx, frameBufferManager, sortFields,
+                            firstKeyNormalizerFactory, comparatorFactories, recordDescriptors[0]);
                     state.frameSorter.reset();
                 }
 
                 @Override
                 public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    state.frameSorter.insertFrame(buffer);
+                    if (!state.frameSorter.insertFrame(buffer)) {
+                        throw new HyracksDataException("Failed to insert the given frame into sorting buffer. "
+                                + "Please increase the sorting memory budget to enable the in-memory sorting, "
+                                + "or you could use ExternalSort instead.");
+                    }
                 }
 
                 @Override
                 public void close() throws HyracksDataException {
-                    state.frameSorter.sortFrames();
+                    state.frameSorter.sort();
                     ctx.setStateObject(state);
                 }
 
@@ -152,7 +165,7 @@ public class InMemorySortOperatorDescriptor extends AbstractOperatorDescriptor {
                     try {
                         SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(
                                 getOperatorId(), SORT_ACTIVITY_ID), partition));
-                        state.frameSorter.flushFrames(writer);
+                        state.frameSorter.flush(writer);
                     } catch (Exception e) {
                         writer.fail();
                         throw new HyracksDataException(e);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
deleted file mode 100644
index ef1ae88..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
+++ /dev/null
@@ -1,218 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.ActivityId;
-import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.TaskId;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.api.job.JobId;
-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.AbstractStateObject;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-
-/**
- * @author pouria
- *         Operator descriptor for sorting with replacement, consisting of two
- *         phases:
- *         - Run Generation: Denoted by OptimizedSortActivity below, in which
- *         sort runs get generated from the input data. This phases uses the
- *         Selection Tree and Memory Manager to benefit from the replacement
- *         selection optimization, to create runs which are longer than the
- *         available memory size.
- *         - Merging: Denoted by MergeActivity below, in which runs (generated
- *         in the previous phase) get merged via a merger. Each run has a single
- *         buffer in memory, and a priority queue is used to select the top
- *         tuple each time. Top tuple is send to a new run or output
- */
-
-public class OptimizedExternalSortOperatorDescriptor extends AbstractOperatorDescriptor {
-
-    private static final int NO_LIMIT = -1;
-    private static final long serialVersionUID = 1L;
-    private static final int SORT_ACTIVITY_ID = 0;
-    private static final int MERGE_ACTIVITY_ID = 1;
-
-    private final int[] sortFields;
-    private final INormalizedKeyComputerFactory firstKeyNormalizerFactory;
-    private final IBinaryComparatorFactory[] comparatorFactories;
-    private final int memSize;
-    private final int outputLimit;
-
-    public OptimizedExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
-            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
-        this(spec, framesLimit, NO_LIMIT, sortFields, null, comparatorFactories, recordDescriptor);
-    }
-
-    public OptimizedExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int outputLimit,
-            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
-        this(spec, framesLimit, outputLimit, sortFields, null, comparatorFactories, recordDescriptor);
-    }
-
-    public OptimizedExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int memSize, int outputLimit,
-            int[] sortFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
-            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
-        super(spec, 1, 1);
-        this.memSize = memSize;
-        this.outputLimit = outputLimit;
-        this.sortFields = sortFields;
-        this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
-        this.comparatorFactories = comparatorFactories;
-        if (memSize <= 1) {
-            throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
-        }
-        recordDescriptors[0] = recordDescriptor;
-    }
-
-    @Override
-    public void contributeActivities(IActivityGraphBuilder builder) {
-        OptimizedSortActivity osa = new OptimizedSortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
-        OptimizedMergeActivity oma = new OptimizedMergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
-
-        builder.addActivity(this, osa);
-        builder.addSourceEdge(0, osa, 0);
-
-        builder.addActivity(this, oma);
-        builder.addTargetEdge(0, oma, 0);
-
-        builder.addBlockingEdge(osa, oma);
-    }
-
-    public static class OptimizedSortTaskState extends AbstractStateObject {
-        private List<IFrameReader> runs;
-
-        public OptimizedSortTaskState() {
-        }
-
-        private OptimizedSortTaskState(JobId jobId, TaskId taskId) {
-            super(jobId, taskId);
-        }
-
-        @Override
-        public void toBytes(DataOutput out) throws IOException {
-
-        }
-
-        @Override
-        public void fromBytes(DataInput in) throws IOException {
-
-        }
-    }
-
-    private class OptimizedSortActivity extends AbstractActivityNode {
-        private static final long serialVersionUID = 1L;
-
-        public OptimizedSortActivity(ActivityId id) {
-            super(id);
-        }
-
-        @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
-            final IRunGenerator runGen;
-            if (outputLimit == NO_LIMIT) {
-                runGen = new OptimizedExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory,
-                        comparatorFactories, recordDescriptors[0], memSize);
-            } else {
-                runGen = new OptimizedExternalSortRunGeneratorWithLimit(ctx, sortFields, firstKeyNormalizerFactory,
-                        comparatorFactories, recordDescriptors[0], memSize, outputLimit);
-            }
-
-            IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
-                @Override
-                public void open() throws HyracksDataException {
-
-                    runGen.open();
-                }
-
-                @Override
-                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    runGen.nextFrame(buffer);
-                }
-
-                @Override
-                public void close() throws HyracksDataException {
-                    OptimizedSortTaskState state = new OptimizedSortTaskState(ctx.getJobletContext().getJobId(),
-                            new TaskId(getActivityId(), partition));
-                    runGen.close();
-                    state.runs = runGen.getRuns();
-                    ctx.setStateObject(state);
-
-                }
-
-                @Override
-                public void fail() throws HyracksDataException {
-                    runGen.fail();
-                }
-            };
-            return op;
-        }
-    }
-
-    private class OptimizedMergeActivity extends AbstractActivityNode {
-        private static final long serialVersionUID = 1L;
-
-        public OptimizedMergeActivity(ActivityId id) {
-            super(id);
-        }
-
-        @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
-            IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
-                @Override
-                public void initialize() throws HyracksDataException {
-                    OptimizedSortTaskState state = (OptimizedSortTaskState) ctx.getStateObject(new TaskId(
-                            new ActivityId(getOperatorId(), SORT_ACTIVITY_ID), partition));
-
-                    List<IFrameReader> runs = state.runs;
-
-                    IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
-                    for (int i = 0; i < comparatorFactories.length; ++i) {
-                        comparators[i] = comparatorFactories[i].createBinaryComparator();
-                    }
-
-                    INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
-                            : firstKeyNormalizerFactory.createNormalizedKeyComputer();
-                    int necessaryFrames = Math.min(runs.size() + 2, memSize);
-                    ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, outputLimit, runs, sortFields,
-                            comparators, nmkComputer, recordDescriptors[0], necessaryFrames, writer);
-
-                    merger.processWithReplacementSelection();
-
-                }
-            };
-            return op;
-        }
-    }
-}
\ No newline at end of file


[04/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetable.java
new file mode 100644
index 0000000..7608128
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetable.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IResetable<T> {
+    void reset(T other);
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparable.java
new file mode 100644
index 0000000..88e2e64
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparable.java
@@ -0,0 +1,19 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IResetableComparable<T> extends IResetable<T>, Comparable<T>{
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparableFactory.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparableFactory.java
new file mode 100644
index 0000000..a5a635e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparableFactory.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IResetableComparableFactory<T> {
+    IResetableComparable<T> createResetableComparable();
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeap.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeap.java
new file mode 100644
index 0000000..1cfb4e0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeap.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public class MaxHeap extends AbstractHeap implements IMaxHeap<IResetableComparable> {
+
+    public MaxHeap(IResetableComparableFactory factory, int capacity) {
+        super(factory, capacity);
+    }
+
+    @Override
+    protected void bubbleUp(int i) {
+        int pid = getParentId(i);
+        if (pid != NOT_EXIST && compareTo(pid, i) < 0) {
+            swap(pid, i);
+            bubbleUp(pid);
+        }
+    }
+
+    @Override
+    protected void trickleDown(int i) {
+        int maxChild = getMaxChild(i);
+        if (maxChild != NOT_EXIST && compareTo(i, maxChild) < 0) {
+            swap(maxChild, i);
+            trickleDown(maxChild);
+        }
+    }
+
+    @Override
+    public void getMax(IResetableComparable result) {
+        result.reset(entries[0]);
+        numEntry--;
+        if (numEntry > 0) {
+            entries[0].reset(entries[numEntry]);
+            trickleDown(0);
+        }
+    }
+
+    @Override
+    public void peekMax(IResetableComparable result) {
+        result.reset(entries[0]);
+    }
+
+    @Override
+    public void replaceMax(IResetableComparable newElement) {
+        entries[0].reset(newElement);
+        trickleDown(0);
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeap.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeap.java
new file mode 100644
index 0000000..bfeda33
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeap.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public class MinHeap extends AbstractHeap implements IMinHeap<IResetableComparable> {
+
+    public MinHeap(IResetableComparableFactory factory, int capacity) {
+        super(factory, capacity);
+    }
+
+    @Override
+    protected void bubbleUp(int i) {
+        int pid = getParentId(i);
+        if (pid != NOT_EXIST && compareTo(pid, i) > 0) {
+            swap(pid, i);
+            bubbleUp(pid);
+        }
+    }
+
+    @Override
+    protected void trickleDown(int i) {
+        int minChild = getMinChild(i);
+        if (minChild != NOT_EXIST && compareTo(i, minChild) > 0) {
+            swap(minChild, i);
+            trickleDown(minChild);
+        }
+    }
+
+    @Override
+    public void getMin(IResetableComparable result) {
+        result.reset(entries[0]);
+        numEntry--;
+        if (numEntry > 0) {
+            entries[0].reset(entries[numEntry]);
+            trickleDown(0);
+        }
+    }
+
+    @Override
+    public void peekMin(IResetableComparable result) {
+        result.reset(entries[0]);
+    }
+
+    @Override
+    public void replaceMin(IResetableComparable newElement) {
+        entries[0].reset(newElement);
+        trickleDown(0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeap.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeap.java
new file mode 100644
index 0000000..e9782e2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeap.java
@@ -0,0 +1,217 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public class MinMaxHeap extends AbstractHeap implements IMinMaxHeap<IResetableComparable> {
+
+    public MinMaxHeap(IResetableComparableFactory factory, int capacity) {
+        super(factory, capacity);
+    }
+
+    @Override
+    protected void bubbleUp(int cid) {
+        int pid = getParentId(cid);
+        if (isAtMinLevel(cid)) {
+            if (pid != NOT_EXIST && entries[pid].compareTo(entries[cid]) < 0) {
+                swap(cid, pid);
+                bubbleUpMax(pid);
+            } else {
+                bubbleUpMin(cid);
+            }
+        } else { // isAtMaxLevel
+            if (pid != NOT_EXIST && entries[pid].compareTo(entries[cid]) > 0) {
+                swap(cid, pid);
+                bubbleUpMin(pid);
+            } else {
+                bubbleUpMax(cid);
+            }
+        }
+    }
+
+    private void bubbleUpMin(int id) {
+        int gp = getGrandParentId(id);
+        if (gp != NOT_EXIST && entries[gp].compareTo(entries[id]) > 0) {
+            swap(gp, id);
+            bubbleUpMin(gp);
+        }
+    }
+
+    private void bubbleUpMax(int id) {
+        int gp = getGrandParentId(id);
+        if (gp != NOT_EXIST && entries[gp].compareTo(entries[id]) < 0) {
+            swap(gp, id);
+            bubbleUpMax(gp);
+        }
+    }
+
+    private boolean isAtMinLevel(int cid) {
+        return getLevel(cid) % 2 == 0;
+    }
+
+    /**
+     * Make sure to check the {@link #isEmpty()} before calling this function.
+     *
+     * @param result
+     */
+    @Override
+    public void getMin(IResetableComparable result) {
+        result.reset(entries[0]);
+        numEntry--;
+        if (numEntry > 0) {
+            entries[0].reset(entries[numEntry]);
+            trickleDown(0);
+        }
+    }
+
+    @Override
+    public void getMax(IResetableComparable result) {
+        int max = getMaxChild(0);
+        if (max == NOT_EXIST) {
+            getMin(result);
+            return;
+        }
+        result.reset(entries[max]);
+        numEntry--;
+        if (numEntry > max) {
+            entries[max].reset(entries[numEntry]);
+            trickleDown(max);
+        }
+    }
+
+    @Override
+    protected void trickleDown(int id) {
+        if (isAtMinLevel(id)) {
+            trickleDownMin(id);
+        } else {
+            trickleDownMax(id);
+        }
+    }
+
+    private void trickleDownMax(int id) {
+        int maxId = getMaxOfDescendents(id);
+        if (maxId == NOT_EXIST) {
+            return;
+        }
+        if (isDirectChild(id, maxId)) {
+            if (entries[id].compareTo(entries[maxId]) < 0) {
+                swap(id, maxId);
+            }
+        } else {
+            if (entries[id].compareTo(entries[maxId]) < 0) {
+                swap(id, maxId);
+                int pid = getParentId(maxId);
+                if (entries[maxId].compareTo(entries[pid]) < 0) {
+                    swap(pid, maxId);
+                }
+                trickleDownMax(maxId);
+            }
+        }
+    }
+
+    private void trickleDownMin(int id) {
+        int minId = getMinOfDescendents(id);
+        if (minId == NOT_EXIST) {
+            return;
+        }
+        if (isDirectChild(id, minId)) {
+            if (entries[id].compareTo(entries[minId]) > 0) {
+                swap(id, minId);
+            }
+        } else { // is grand child
+            if (entries[id].compareTo(entries[minId]) > 0) {
+                swap(id, minId);
+                int pid = getParentId(minId);
+                if (entries[minId].compareTo(entries[pid]) > 0) {
+                    swap(pid, minId);
+                }
+                trickleDownMin(minId);
+            }
+        }
+    }
+
+    private int getMaxOfDescendents(int id) {
+        int max = getMaxChild(id);
+        if (max != NOT_EXIST) {
+            int leftMax = getMaxChild(getLeftChild(id));
+            if (leftMax != NOT_EXIST) {
+                max = entries[leftMax].compareTo(entries[max]) > 0 ? leftMax : max;
+                int rightMax = getMaxChild(getRightChild(id));
+                if (rightMax != NOT_EXIST) {
+                    max = entries[rightMax].compareTo(entries[max]) > 0 ? rightMax : max;
+                }
+            }
+        }
+        return max;
+    }
+
+    private int getMinOfDescendents(int id) {
+        int min = getMinChild(id);
+        if (min != NOT_EXIST) {
+            int leftMin = getMinChild(getLeftChild(id));
+            if (leftMin != NOT_EXIST) {
+                min = entries[leftMin].compareTo(entries[min]) < 0 ? leftMin : min;
+                int rightMin = getMinChild(getRightChild(id));
+                if (rightMin != NOT_EXIST) {
+                    min = entries[rightMin].compareTo(entries[min]) < 0 ? rightMin : min;
+                }
+            }
+        }
+        return min;
+    }
+
+    public boolean isEmpty() {
+        return numEntry == 0;
+    }
+
+    /**
+     * Make sure to call the {@link #isEmpty()} before calling this function
+     *
+     * @param result is the object that will eventually contain minimum entry
+     */
+    @Override
+    public void peekMin(IResetableComparable result) {
+        result.reset(entries[0]);
+    }
+
+    @Override
+    public void peekMax(IResetableComparable result) {
+        int maxChild = getMaxChild(0);
+        if (maxChild == NOT_EXIST) {
+            peekMin(result);
+            return;
+        }
+        result.reset(entries[maxChild]);
+    }
+
+    @Override
+    public void replaceMin(IResetableComparable newElement) {
+        entries[0].reset(newElement);
+        trickleDown(0);
+    }
+
+    @Override
+    public void replaceMax(IResetableComparable newElement) {
+        int maxChild = getMaxChild(0);
+        if (maxChild == NOT_EXIST) {
+            replaceMin(newElement);
+            return;
+        }
+        entries[maxChild].reset(newElement);
+        bubbleUp(maxChild);
+        trickleDown(maxChild);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
index 1ea8393..3837004 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
@@ -41,7 +41,7 @@ public class SerializableHashTable implements ISerializableTable {
 
     public SerializableHashTable(int tableSize, final IHyracksTaskContext ctx) throws HyracksDataException {
         this.ctx = ctx;
-        int frameSize = ctx.getFrameSize();
+        int frameSize = ctx.getInitialFrameSize();
 
         int residual = tableSize * INT_SIZE * 2 % frameSize == 0 ? 0 : 1;
         int headerSize = tableSize * INT_SIZE * 2 / frameSize + residual;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/TuplePointer.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/TuplePointer.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/TuplePointer.java
index 8ef478c..a42e06d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/TuplePointer.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/TuplePointer.java
@@ -14,7 +14,48 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.structures;
 
-public class TuplePointer {
+public class TuplePointer implements IResetable<TuplePointer> {
+    public static final int INVALID_ID = -1;
     public int frameIndex;
     public int tupleIndex;
+
+    public TuplePointer() {
+        this(INVALID_ID, INVALID_ID);
+    }
+
+    public TuplePointer(int frameId, int tupleId) {
+        reset(frameId, tupleId);
+    }
+
+    public void reset(TuplePointer other) {
+        reset(other.frameIndex, other.tupleIndex);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        TuplePointer that = (TuplePointer) o;
+
+        if (frameIndex != that.frameIndex)
+            return false;
+        return tupleIndex == that.tupleIndex;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = frameIndex;
+        result = 31 * result + tupleIndex;
+        return result;
+    }
+
+    public void reset(int frameId, int tupleId) {
+        this.frameIndex = frameId;
+        this.tupleIndex = tupleId;
+    }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
index aff4273..09aacd6 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
@@ -36,7 +36,7 @@ public final class DeserializedOperatorNodePushable extends AbstractUnaryInputOp
             RecordDescriptor inRecordDesc) {
         this.ctx = ctx;
         this.delegate = delegate;
-        deserializer = inRecordDesc == null ? null : new FrameDeserializer(ctx.getFrameSize(), inRecordDesc);
+        deserializer = inRecordDesc == null ? null : new FrameDeserializer(inRecordDesc);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/MathUtil.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/MathUtil.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/MathUtil.java
new file mode 100644
index 0000000..c89ee25
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/MathUtil.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.util;
+
+public class MathUtil {
+    /**
+     * Fast way to calculate the log2(x). Note: x should be >= 1.
+     *
+     * @param n
+     * @return
+     */
+    public static int log2Floor(int n) {
+        assert n >= 1;
+        int log = 0;
+        if (n > 0xffff) {
+            n >>>= 16;
+            log = 16;
+        }
+
+        if (n > 0xff) {
+            n >>>= 8;
+            log |= 8;
+        }
+
+        if (n > 0xf) {
+            n >>>= 4;
+            log |= 4;
+        }
+
+        if (n > 0b11) {
+            n >>>= 2;
+            log |= 2;
+        }
+
+        return log + (n >>> 1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
index c06b50c..d6adcf4 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
@@ -14,12 +14,13 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.util;
 
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public class ReferenceEntry {
     private final int runid;
-    private FrameTupleAccessor acccessor;
+    private IFrameTupleAccessor acccessor;
     private int tupleIndex;
     private int[] tPointers;
 
@@ -38,11 +39,11 @@ public class ReferenceEntry {
         return runid;
     }
 
-    public FrameTupleAccessor getAccessor() {
+    public IFrameTupleAccessor getAccessor() {
         return acccessor;
     }
 
-    public void setAccessor(FrameTupleAccessor fta) {
+    public void setAccessor(IFrameTupleAccessor fta) {
         this.acccessor = fta;
     }
 
@@ -62,15 +63,14 @@ public class ReferenceEntry {
         initTPointer(acccessor, tupleIndex, keyFields, nmkComputer);
     }
 
-    private void initTPointer(FrameTupleAccessor fta, int tupleIndex, int[] keyFields,
+    private void initTPointer(IFrameTupleAccessor fta, int tupleIndex, int[] keyFields,
             INormalizedKeyComputer nmkComputer) {
         this.tupleIndex = tupleIndex;
         byte[] b1 = fta.getBuffer().array();
         for (int f = 0; f < keyFields.length; ++f) {
             int fIdx = keyFields[f];
-            tPointers[2 * f + 1] = fta.getTupleStartOffset(tupleIndex) + fta.getFieldSlotsLength()
-                    + fta.getFieldStartOffset(tupleIndex, fIdx);
-            tPointers[2 * f + 2] = fta.getFieldEndOffset(tupleIndex, fIdx) - fta.getFieldStartOffset(tupleIndex, fIdx);
+            tPointers[2 * f + 1] = fta.getAbsoluteFieldStartOffset(tupleIndex, fIdx);
+            tPointers[2 * f + 2] = fta.getFieldLength(tupleIndex, fIdx);
             if (f == 0) {
                 if (nmkComputer != null) {
                     tPointers[0] = nmkComputer.normalize(b1, tPointers[1], tPointers[2]);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
index 225f583..e994f04 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
@@ -18,13 +18,10 @@ import java.io.IOException;
 import java.util.BitSet;
 import java.util.Comparator;
 
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 
 public class ReferencedPriorityQueue {
-    private final int frameSize;
-    private final RecordDescriptor recordDescriptor;
     private final ReferenceEntry entries[];
     private final int size;
     private final BitSet runAvail;
@@ -34,10 +31,8 @@ public class ReferencedPriorityQueue {
     private final INormalizedKeyComputer nmkComputer;
     private final int[] keyFields;
 
-    public ReferencedPriorityQueue(int frameSize, RecordDescriptor recordDescriptor, int initSize,
-            Comparator<ReferenceEntry> comparator, int[] keyFields, INormalizedKeyComputer nmkComputer) {
-        this.frameSize = frameSize;
-        this.recordDescriptor = recordDescriptor;
+    public ReferencedPriorityQueue(int initSize, Comparator<ReferenceEntry> comparator, int[] keyFields,
+            INormalizedKeyComputer nmkComputer) {
         if (initSize < 1)
             throw new IllegalArgumentException();
         this.comparator = comparator;
@@ -55,7 +50,7 @@ public class ReferencedPriorityQueue {
 
     /**
      * Retrieve the top entry without removing it
-     * 
+     *
      * @return the top entry
      */
     public ReferenceEntry peek() {
@@ -65,17 +60,14 @@ public class ReferencedPriorityQueue {
     /**
      * compare the new entry with entries within the queue, to find a spot for
      * this new entry
-     * 
-     * @param entry
+     *
+     * @param fta
      * @return runid of this entry
      * @throws IOException
      */
-    public int popAndReplace(FrameTupleAccessor fta, int tIndex) {
+    public int popAndReplace(IFrameTupleAccessor fta, int tIndex) {
         ReferenceEntry entry = entries[0];
-        if (entry.getAccessor() == null) {
-            entry.setAccessor(new FrameTupleAccessor(frameSize, recordDescriptor));
-        }
-        entry.getAccessor().reset(fta.getBuffer());
+        entry.setAccessor(fta);
         entry.setTupleIndex(tIndex, keyFields, nmkComputer);
 
         add(entry);
@@ -84,9 +76,8 @@ public class ReferencedPriorityQueue {
 
     /**
      * Push entry into priority queue
-     * 
-     * @param e
-     *            the new Entry
+     *
+     * @param e the new Entry
      */
     private void add(ReferenceEntry e) {
         ReferenceEntry min = entries[0];
@@ -127,7 +118,7 @@ public class ReferencedPriorityQueue {
 
     /**
      * Pop is called only when a run is exhausted
-     * 
+     *
      * @return
      */
     public ReferenceEntry pop() {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/Utility.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/Utility.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/Utility.java
new file mode 100644
index 0000000..ee2b008
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/Utility.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+public class Utility {
+
+    public static String repeatString(char ch, int times) {
+        return new String(new char[times]).replace('\0', ch);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/Common.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/Common.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/Common.java
new file mode 100644
index 0000000..5a59ab6
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/Common.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
+
+public class Common {
+    static int MIN_FRAME_SIZE = 256;
+    static int NUM_MIN_FRAME = 15;
+    static int BUDGET = NUM_MIN_FRAME * MIN_FRAME_SIZE;
+
+    static FrameManager commonFrameManager = new FrameManager(MIN_FRAME_SIZE);
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java
new file mode 100644
index 0000000..a2d0f1e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class FrameFreeSlotBestFitUsingTreeMapTest {
+
+    static int size = 10;
+
+    FrameFreeSlotSmallestFit policy;
+
+    @Before
+    public void intial() {
+        policy = new FrameFreeSlotSmallestFit();
+    }
+
+    @Test
+    public void testAll() {
+
+        for (int i = 0; i < size; i++) {
+            policy.pushNewFrame(i, i);
+            assertEquals(i, policy.popBestFit(i));
+        }
+        assertEquals(-1, policy.popBestFit(0));
+
+        for (int i = 0; i < size; i++) {
+            policy.pushNewFrame(i, i);
+        }
+        for (int i = 0; i < size; i++) {
+            assertEquals(i, policy.popBestFit(i));
+        }
+
+    }
+
+    @Test
+    public void testReset(){
+        testAll();
+        policy.reset();
+        testAll();
+    }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirstTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirstTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirstTest.java
new file mode 100644
index 0000000..f3b923e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirstTest.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static junit.framework.Assert.assertEquals;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class FrameFreeSlotBiggestFirstTest {
+
+    static int size = 10;
+
+    FrameFreeSlotBiggestFirst policy;
+
+    @Before
+    public void intial() {
+        policy = new FrameFreeSlotBiggestFirst(size);
+    }
+
+    @Test
+    public void testAll() {
+
+        for (int i = 0; i < size; i++) {
+            policy.pushNewFrame(i, i);
+            assertEquals(i, policy.popBestFit(i));
+        }
+        assertEquals(-1, policy.popBestFit(0));
+
+        for (int i = 0; i < size; i++) {
+            policy.pushNewFrame(i, i);
+        }
+        for (int i = 0; i < size; i++) {
+            assertEquals(size - i - 1, policy.popBestFit(0));
+        }
+
+        for (int i = 0; i < size; i++) {
+            policy.pushNewFrame(i, i);
+        }
+        for (int i = 0; i < size / 2; i++) {
+            assertEquals(size - i - 1, policy.popBestFit(size / 2));
+        }
+        assertEquals(-1, policy.popBestFit(size / 2));
+        for (int i = 0; i < size / 2; i++) {
+            assertEquals(size / 2 - i - 1, policy.popBestFit(0));
+        }
+
+    }
+
+    @Test
+    public void testReset() {
+        testAll();
+        policy.reset();
+        testAll();
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFitTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFitTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFitTest.java
new file mode 100644
index 0000000..94a8493
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFitTest.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class FrameFreeSlotLastFitTest {
+
+    FrameFreeSlotLastFit zeroPolicy;
+    FrameFreeSlotLastFit unifiedPolicy;
+    FrameFreeSlotLastFit ascPolicy;
+    FrameFreeSlotLastFit dscPolicy;
+
+    static final int size = 10;
+    static final int medium = 5;
+
+    @Before
+    public void setUp() throws Exception {
+        zeroPolicy = new FrameFreeSlotLastFit(0);
+        unifiedPolicy = new FrameFreeSlotLastFit(size);
+        ascPolicy = new FrameFreeSlotLastFit(size);
+        dscPolicy = new FrameFreeSlotLastFit(size);
+    }
+
+    @Test
+    public void testPushAndPop() throws Exception {
+        for (int i = 0; i < size; i++) {
+            unifiedPolicy.pushNewFrame(i, medium);
+        }
+        for (int i = 0; i < size; i++) {
+            assertTrue(unifiedPolicy.popBestFit(medium) == size - i - 1);
+        }
+        assertTrue(unifiedPolicy.popBestFit(0) == -1);
+
+        for (int i = 0; i < size / 2; i++) {
+            ascPolicy.pushNewFrame(i, i);
+            assertEquals(ascPolicy.popBestFit(medium), -1);
+            dscPolicy.pushNewFrame(i, size - i - 1);
+            assertEquals(dscPolicy.popBestFit(medium), i);
+        }
+
+        for (int i = size / 2; i < size; i++) {
+            ascPolicy.pushNewFrame(i, i);
+            assertEquals(ascPolicy.popBestFit(medium), i);
+            dscPolicy.pushNewFrame(i, size - i - 1);
+            assertEquals(dscPolicy.popBestFit(medium), -1);
+        }
+
+        ascPolicy.reset();
+        for (int i = 0; i < size; i++) {
+            ascPolicy.pushNewFrame(size - i, size - i);
+        }
+
+        for (int i = 0; i < size; i++) {
+            assertEquals(size - i, ascPolicy.popBestFit(size - i));
+        }
+    }
+
+    @Test
+    public void testReset() throws Exception {
+        testPushAndPop();
+
+        zeroPolicy.reset();
+        unifiedPolicy.reset();
+        ascPolicy.reset();
+        dscPolicy.reset();
+        testPushAndPop();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePoolTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePoolTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePoolTest.java
new file mode 100644
index 0000000..e9ac2ec
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePoolTest.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.BUDGET;
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.MIN_FRAME_SIZE;
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.NUM_MIN_FRAME;
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.commonFrameManager;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class VariableFramePoolTest {
+
+    VariableFramePool pool;
+    @Before
+    public void setUp() throws Exception {
+
+        pool = new VariableFramePool(commonFrameManager, BUDGET);
+    }
+
+    @Test
+    public void testGetMinFrameSize() throws Exception {
+        assertEquals(MIN_FRAME_SIZE, commonFrameManager.getInitialFrameSize());
+        assertEquals(MIN_FRAME_SIZE, pool.getMinFrameSize());
+    }
+
+    @Test
+    public void testGetMemoryBudgetBytes() throws Exception {
+        assertEquals(BUDGET, pool.getMemoryBudgetBytes());
+    }
+
+    @Test
+    public void testAllocateUniformFrameShouldSuccess() throws Exception {
+        testAllocateAllSpacesWithMinFrames();
+        testAllocateShouldFailAfterAllSpaceGetUsed();
+        pool.reset();
+        testAllocateAllSpacesWithMinFrames();
+        pool.close();
+    }
+
+    @Test
+    public void testResetShouldReuseExistingFrames() throws HyracksDataException {
+        Set<?> set1 = testAllocateAllSpacesWithMinFrames();
+        pool.reset();
+        Set<?> set2 = testAllocateAllSpacesWithMinFrames();
+        assertEquals(set1, set2);
+        pool.close();
+    }
+
+    @Test
+    public void testCloseShouldNotReuseExistingFrames() throws HyracksDataException {
+        Set<?> set1 = testAllocateAllSpacesWithMinFrames();
+        pool.close();
+        Set<?> set2 = testAllocateAllSpacesWithMinFrames();
+        assertFalse(set1.equals(set2));
+        pool.close();
+    }
+
+    @Test
+    public void testShouldReturnLargerFramesIfFitOneIsUsed() throws HyracksDataException {
+        Set<?> set = testAllocateVariableFrames();
+        pool.reset();
+        testShouldFindTheMatchFrames(set);
+        pool.reset();
+
+        // allocate seq: 1, 1, 2, 3, 4
+        ByteBuffer placeBuffer = pool.allocateFrame(MIN_FRAME_SIZE);
+        assertTrue(set.contains(new ByteBufferPtr(placeBuffer)));
+        for (int i = 1; i <= 4 ; i++) {
+            ByteBuffer buffer = pool.allocateFrame(i * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(set.contains(new ByteBufferPtr(buffer)));
+        }
+        assertNull(pool.allocateFrame(MIN_FRAME_SIZE));
+        pool.close();
+    }
+
+    @Test
+    public void testShouldMergeIfNoLargerFrames() throws HyracksDataException {
+        Set<?> set = testAllocateAllSpacesWithMinFrames();
+        pool.reset();
+        int chunks = 5;
+        for (int i = 0; i < NUM_MIN_FRAME; i+= chunks) {
+            ByteBuffer buffer = pool.allocateFrame(chunks * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(!set.contains(new ByteBufferPtr(buffer)));
+        }
+    }
+
+    @Test
+    public void testUseMiddleSizeFrameAndNeedToMergeSmallAndBigger() throws HyracksDataException {
+        Set<?> set = testAllocateVariableFrames();
+        pool.reset();
+        // allocate seq: 3, 6, 1;
+        ByteBuffer buffer = pool.allocateFrame(3 * MIN_FRAME_SIZE);
+        assertTrue(set.contains(new ByteBufferPtr(buffer)));
+        buffer = pool.allocateFrame(6 * MIN_FRAME_SIZE);
+        assertFalse(set.contains(new ByteBufferPtr(buffer)));
+        buffer = pool.allocateFrame(1 * MIN_FRAME_SIZE);
+        assertTrue(set.contains(new ByteBufferPtr(buffer)));
+        assertEquals(5 * MIN_FRAME_SIZE, buffer.capacity());
+        pool.reset();
+    }
+
+    private void testAllocateShouldFailAfterAllSpaceGetUsed() throws HyracksDataException {
+        for (int i = 0; i < NUM_MIN_FRAME; i++) {
+            assertNull(pool.allocateFrame(MIN_FRAME_SIZE));
+        }
+    }
+
+    private HashSet<ByteBufferPtr> testAllocateAllSpacesWithMinFrames() throws HyracksDataException {
+        HashSet<ByteBufferPtr> set = new HashSet<>();
+        for (int i = 0; i < NUM_MIN_FRAME; i++) {
+            ByteBuffer buffer = pool.allocateFrame(MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(!set.contains(new ByteBufferPtr(buffer)));
+            set.add(new ByteBufferPtr(buffer));
+        }
+        return set;
+    }
+
+    /**
+     * Pool will become 1,2,3,4,5
+     *
+     * @throws HyracksDataException
+     */
+    private Set<ByteBufferPtr> testAllocateVariableFrames() throws HyracksDataException {
+        int budget = BUDGET;
+        int allocate = 0;
+        int i = 1;
+        Set<ByteBufferPtr> set = new HashSet<>();
+        while (budget - allocate >= i * MIN_FRAME_SIZE) {
+            ByteBuffer buffer = pool.allocateFrame(i * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            set.add(new ByteBufferPtr(buffer));
+            allocate += i++ * MIN_FRAME_SIZE;
+        }
+        return set;
+    }
+
+    private void testShouldFindTheMatchFrames(Set<?> set) throws HyracksDataException {
+        pool.reset();
+        List<Integer> list = Arrays.asList(1, 2, 3, 4, 5);
+
+        for (int i = 0; i < list.size(); i++) {
+            ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(set.contains(new ByteBufferPtr(buffer)));
+            assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
+        }
+        pool.reset();
+        for (int i = list.size() - 1; i >= 0; i--) {
+            ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(set.contains(new ByteBufferPtr(buffer)));
+            assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
+        }
+
+        Collections.shuffle(list);
+        pool.reset();
+        for (int i = 0; i < list.size(); i++) {
+            ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
+            assertNotNull(buffer);
+            assertTrue(set.contains(new ByteBufferPtr(buffer)));
+            assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
+        }
+
+    }
+
+    public static class ByteBufferPtr {
+        ByteBuffer bytebuffer;
+
+        public ByteBufferPtr(ByteBuffer buffer) {
+            bytebuffer = buffer;
+        }
+
+        @Override
+        public int hashCode() {
+            return bytebuffer.hashCode();
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            return this.bytebuffer == ((ByteBufferPtr) obj).bytebuffer;
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramesMemoryManagerTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramesMemoryManagerTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramesMemoryManagerTest.java
new file mode 100644
index 0000000..f56a62d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramesMemoryManagerTest.java
@@ -0,0 +1,170 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.BUDGET;
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.MIN_FRAME_SIZE;
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.NUM_MIN_FRAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.FixedSizeFrame;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+
+public class VariableFramesMemoryManagerTest {
+    VariableFrameMemoryManager framesMemoryManager;
+    FrameTupleAccessor fta;
+    Random random;
+    List<IFrame> frameList;
+
+    @Before
+    public void setUp() throws Exception {
+        VariableFramePool framePool = new VariableFramePool(Common.commonFrameManager, BUDGET);
+        FrameFreeSlotLastFit policy = new FrameFreeSlotLastFit(NUM_MIN_FRAME);
+        framesMemoryManager = new VariableFrameMemoryManager(framePool, policy);
+        RecordDescriptor recordDescriptor = new RecordDescriptor(new ISerializerDeserializer[] { null });
+        fta = new FrameTupleAccessor(recordDescriptor);
+        random = new Random(System.currentTimeMillis());
+        frameList = new ArrayList<>();
+    }
+
+    @Test
+    public void testNormalIncomingFrames() throws HyracksDataException {
+        HashMap<Integer, Integer> tupleSet = prepareTuples();
+        for (IFrame frame : frameList) {
+            assertTrue(framesMemoryManager.insertFrame(frame.getBuffer()) >=0);
+        }
+        assertEquals(NUM_MIN_FRAME, framesMemoryManager.getNumFrames());
+        assertEveryTupleInFTAIsInFrameMemoryManager(tupleSet, framesMemoryManager);
+    }
+
+    @Test
+    public void testRandomTuplesAreAllStoredInBuffer() throws HyracksDataException {
+        Map<Integer, Integer> tupleSet = prepareRandomTuples();
+        for (IFrame frame : frameList) {
+            if (framesMemoryManager.insertFrame(frame.getBuffer()) < 0) {
+                fta.reset(frame.getBuffer());
+                for (int i = 0; i < fta.getTupleCount(); ++i) {
+                    int id = parseTuple(fta.getBuffer(),
+                            fta.getTupleStartOffset(i) + fta.getFieldStartOffset(i, 0) + fta.getFieldSlotsLength());
+                    tupleSet.remove(id);
+                    //                    System.out.println(
+                    //                            "can't appended id:" + id + ",frameSize:" + frame.getInitialFrameSize());
+                }
+            }
+        }
+        assertEveryTupleInFTAIsInFrameMemoryManager(tupleSet, framesMemoryManager);
+        framesMemoryManager.reset();
+    }
+
+    @Test
+    public void testResetShouldWork() throws HyracksDataException {
+        testNormalIncomingFrames();
+        framesMemoryManager.reset();
+        testRandomTuplesAreAllStoredInBuffer();
+        framesMemoryManager.reset();
+        testRandomTuplesAreAllStoredInBuffer();
+    }
+
+    @Test
+    public void testCloseShouldAlsoWork() throws HyracksDataException {
+        testRandomTuplesAreAllStoredInBuffer();
+        framesMemoryManager.close();
+        testRandomTuplesAreAllStoredInBuffer();
+        framesMemoryManager.close();
+        testRandomTuplesAreAllStoredInBuffer();
+    }
+
+    private HashMap<Integer, Integer> prepareRandomTuples() throws HyracksDataException {
+        frameList.clear();
+        HashMap<Integer, Integer> set = new HashMap<>(NUM_MIN_FRAME);
+        int[] fieldSlot = { 0 };
+        int id = 0;
+        int size = 0;
+        while (size < BUDGET) {
+            int tupleLength = random.nextInt(BUDGET / 3) + 4;
+            IFrame frame = new FixedSizeFrame(Common.commonFrameManager
+                    .allocateFrame(FrameHelper.calcAlignedFrameSizeToStore(1, tupleLength, MIN_FRAME_SIZE)));
+            IFrameTupleAppender appender = new FrameTupleAppender();
+            appender.reset(frame, true);
+            //            System.out.println("id:" + id + ",frameSize:" + frame.getInitialFrameSize() / MIN_FRAME_SIZE);
+            ByteBuffer buffer = ByteBuffer.allocate(tupleLength);
+            buffer.putInt(0, id);
+            assertTrue(appender.append(fieldSlot, buffer.array(), 0, buffer.capacity()));
+            set.put(id++, tupleLength);
+            size += frame.getFrameSize();
+            frameList.add(frame);
+        }
+        return set;
+    }
+
+    private HashMap<Integer, Integer> prepareTuples() throws HyracksDataException {
+        frameList.clear();
+        HashMap<Integer, Integer> set = new HashMap<>(NUM_MIN_FRAME);
+        for (int i = 0; i < NUM_MIN_FRAME; ++i) {
+            IFrame frame = new FixedSizeFrame(Common.commonFrameManager.allocateFrame(MIN_FRAME_SIZE));
+            IFrameTupleAppender appender = new FrameTupleAppender();
+            appender.reset(frame, true);
+
+            int[] fieldSlot = { 0 };
+            ByteBuffer buffer = ByteBuffer.allocate(MIN_FRAME_SIZE / 2);
+            buffer.putInt(0, i);
+            appender.append(fieldSlot, buffer.array(), 0, buffer.capacity());
+            set.put(i, buffer.capacity());
+            frameList.add(frame);
+        }
+        return set;
+    }
+
+    private void assertEveryTupleInFTAIsInFrameMemoryManager(Map<Integer, Integer> tupleSet,
+            VariableFrameMemoryManager framesMemoryManager) {
+        for (int i = 0; i < framesMemoryManager.getNumFrames(); ++i) {
+            fta.reset(framesMemoryManager.getFrame(i), framesMemoryManager.getFrameStartOffset(i),
+                    framesMemoryManager.getFrameSize(i));
+            for (int t = 0; t < fta.getTupleCount(); t++) {
+                int id = parseTuple(fta.getBuffer(), fta.getTupleStartOffset(t) + fta.getFieldSlotsLength() + fta
+                        .getFieldStartOffset(t, 0));
+                //                System.out.println("frameid:" + i + ",tuple:" + t + ",has id:" + id + ",length:" +
+                //                        (fta.getTupleEndOffset(t) - fta.getTupleStartOffset(t) - fta.getFieldSlotsLength()));
+                assertTrue(tupleSet.remove(id) == fta.getTupleEndOffset(t) - fta.getTupleStartOffset(t) - fta
+                        .getFieldSlotsLength());
+            }
+        }
+        assertTrue(tupleSet.isEmpty());
+    }
+
+    private int parseTuple(ByteBuffer buffer, int fieldStartOffset) {
+        return buffer.getInt(fieldStartOffset);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
new file mode 100644
index 0000000..0c54607
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
@@ -0,0 +1,230 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Random;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.FixedSizeFrame;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.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.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.Utility;
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public class VariableTupleMemoryManagerTest {
+    ISerializerDeserializer[] fieldsSerDer = new ISerializerDeserializer[] {
+            IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+    RecordDescriptor recordDescriptor = new RecordDescriptor(fieldsSerDer);
+    ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
+    VariableTupleMemoryManager tupleMemoryManager;
+    FrameTupleAccessor inFTA = new FrameTupleAccessor(recordDescriptor);
+    Random random = new Random(System.currentTimeMillis());
+
+    @Before
+    public void setup() {
+        VariableFramePool framePool = new VariableFramePool(Common.commonFrameManager, Common.BUDGET);
+        tupleMemoryManager = new VariableTupleMemoryManager(framePool, recordDescriptor);
+    }
+
+    @Test
+    public void testInsertTupleToMemoryManager() throws HyracksDataException {
+        int iTuplePerFrame = 3;
+        Map<Integer, Integer> mapPrepare = prepareFixedSizeTuples(iTuplePerFrame);
+        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
+        assertEachTupleInFTAIsInBuffer(mapPrepare, mapInserted);
+    }
+
+    @Test
+    public void testReset() throws HyracksDataException {
+        testInsertVariableSizeTupleToMemoryManager();
+        tupleMemoryManager.reset();
+        testInsertTupleToMemoryManager();
+        tupleMemoryManager.reset();
+        testInsertVariableSizeTupleToMemoryManager();
+    }
+
+    @Test
+    public void testDeleteTupleInMemoryManager() throws HyracksDataException {
+        int iTuplePerFrame = 3;
+        Map<Integer, Integer> map = prepareFixedSizeTuples(iTuplePerFrame);
+        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
+        deleteRandomSelectedTuples(map, mapInserted, 1);
+        assertEachTupleInFTAIsInBuffer(map, mapInserted);
+    }
+
+    @Test
+    public void testReOrganizeSpace() throws HyracksDataException {
+        int iTuplePerFrame = 3;
+        Map<Integer, Integer> map = prepareFixedSizeTuples(iTuplePerFrame);
+        Map<Integer, Integer> copyMap = new HashMap<>(map);
+        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
+        ByteBuffer buffer = deleteRandomSelectedTuples(map, mapInserted, map.size() / 2);
+        inFTA.reset(buffer);
+        Map<TuplePointer, Integer> mapInserted2 = insertInFTAToBufferShouldAllSuccess();
+        Map<TuplePointer, Integer> mergedMap = new HashMap<>(mapInserted);
+        mergedMap.putAll(mapInserted2);
+        assertEachTupleInFTAIsInBuffer(copyMap, mergedMap);
+    }
+
+    @Test
+    public void testReOrganizeVariableSizeTuple() throws HyracksDataException {
+        Map<Integer, Integer> map = prepareVariableSizeTuples();
+        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferCouldFailForLargerTuples(map);
+        Map<Integer, Integer> copyMap = new HashMap<>(map);
+
+        ByteBuffer buffer = deleteRandomSelectedTuples(map, mapInserted, map.size() / 2);
+        inFTA.reset(buffer);
+
+        Map<TuplePointer, Integer> mapInserted2 = insertInFTAToBufferCouldFailForLargerTuples(copyMap);
+        Map<TuplePointer, Integer> mergedMap = new HashMap<>(mapInserted);
+        mergedMap.putAll(mapInserted2);
+
+        assertEachTupleInFTAIsInBuffer(copyMap, mergedMap);
+    }
+
+    @Test
+    public void testInsertVariableSizeTupleToMemoryManager() throws HyracksDataException {
+        Map<Integer, Integer> map = prepareVariableSizeTuples();
+        Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferCouldFailForLargerTuples(map);
+        assertEachTupleInFTAIsInBuffer(map, mapInserted);
+    }
+
+    private void assertEachTupleInFTAIsInBuffer(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted) {
+        ITupleBufferAccessor accessor = tupleMemoryManager.getTupleAccessor();
+        for (Map.Entry<TuplePointer, Integer> entry : mapInserted.entrySet()) {
+            accessor.reset(entry.getKey());
+            int dataLength = map.get(entry.getValue());
+            assertEquals((int) entry.getValue(),
+                    IntSerDeUtils.getInt(accessor.getTupleBuffer().array(), accessor.getAbsFieldStartOffset(0)));
+            assertEquals(dataLength, accessor.getTupleLength());
+        }
+        assertEquals(map.size(), mapInserted.size());
+    }
+
+    private Map<Integer, Integer> prepareFixedSizeTuples(int tuplePerFrame) throws HyracksDataException {
+        Map<Integer, Integer> dataSet = new HashMap<>();
+        ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
+        FixedSizeFrame frame = new FixedSizeFrame(buffer);
+        FrameTupleAppender appender = new FrameTupleAppender();
+        appender.reset(frame, true);
+
+        int sizePerTuple = (Common.MIN_FRAME_SIZE - 1 - 4 - tuplePerFrame * 4 - 4) / tuplePerFrame;
+        int sizeChar = sizePerTuple - fieldsSerDer.length * 4 - 4 - 4;
+        assert (sizeChar > 0);
+        for (int i = 0; i < Common.NUM_MIN_FRAME * tuplePerFrame; i++) {
+            tupleBuilder.reset();
+            tupleBuilder.addField(fieldsSerDer[0], i);
+            tupleBuilder.addField(fieldsSerDer[1], Utility.repeatString('a', sizeChar));
+            assertTrue(appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+                    tupleBuilder.getSize()));
+            dataSet.put(i, tupleBuilder.getSize() + tupleBuilder.getFieldEndOffsets().length * 4);
+        }
+        inFTA.reset(buffer);
+        return dataSet;
+    }
+
+    private Map<Integer, Integer> prepareVariableSizeTuples() throws HyracksDataException {
+        Map<Integer, Integer> dataSet = new HashMap<>();
+        ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
+        FixedSizeFrame frame = new FixedSizeFrame(buffer);
+        FrameTupleAppender appender = new FrameTupleAppender();
+        appender.reset(frame, true);
+
+        for (int i = 0; true; i++) {
+            tupleBuilder.reset();
+            tupleBuilder.addField(fieldsSerDer[0], i);
+            tupleBuilder.addField(fieldsSerDer[1], Utility.repeatString('a', i));
+            if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+                    tupleBuilder.getSize())) {
+                break;
+            }
+            dataSet.put(i, tupleBuilder.getSize() + tupleBuilder.getFieldEndOffsets().length * 4);
+        }
+        inFTA.reset(buffer);
+        return dataSet;
+    }
+
+    private Map<TuplePointer, Integer> insertInFTAToBufferShouldAllSuccess() throws HyracksDataException {
+        Map<TuplePointer, Integer> tuplePointerIntegerMap = new HashMap<>();
+        for (int i = 0; i < inFTA.getTupleCount(); i++) {
+            TuplePointer tuplePointer = new TuplePointer();
+            assertTrue(tupleMemoryManager.insertTuple(inFTA, i, tuplePointer));
+            tuplePointerIntegerMap.put(tuplePointer,
+                    IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+        }
+        return tuplePointerIntegerMap;
+    }
+
+    private Map<TuplePointer, Integer> insertInFTAToBufferCouldFailForLargerTuples(Map<Integer, Integer> map)
+            throws HyracksDataException {
+        Map<TuplePointer, Integer> tuplePointerIdMap = new HashMap<>();
+        int i = 0;
+        for (; i < inFTA.getTupleCount(); i++) {
+            TuplePointer tuplePointer = new TuplePointer();
+            if (!tupleMemoryManager.insertTuple(inFTA, i, tuplePointer)) {
+                break;
+            }
+            tuplePointerIdMap.put(tuplePointer,
+                    IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+        }
+        for (; i < inFTA.getTupleCount(); i++) {
+            map.remove(IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+        }
+        return tuplePointerIdMap;
+    }
+
+    private ByteBuffer deleteRandomSelectedTuples(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted,
+            int minNumOfRecordTobeDeleted)
+            throws HyracksDataException {
+        ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
+        FixedSizeFrame frame = new FixedSizeFrame(buffer);
+        FrameTupleAppender appender = new FrameTupleAppender();
+        appender.reset(frame, true);
+
+        assert (minNumOfRecordTobeDeleted < mapInserted.size());
+        int countDeleted = minNumOfRecordTobeDeleted + random.nextInt(mapInserted.size() - minNumOfRecordTobeDeleted);
+
+        ITupleBufferAccessor accessor = tupleMemoryManager.getTupleAccessor();
+        for (int i = 0; i < countDeleted; i++) {
+            Iterator<Map.Entry<TuplePointer, Integer>> iter = mapInserted.entrySet().iterator();
+            assert (iter.hasNext());
+            Map.Entry<TuplePointer, Integer> pair = iter.next();
+            accessor.reset(pair.getKey());
+            appender.append(accessor.getTupleBuffer().array(), accessor.getTupleStartOffset(),
+                    accessor.getTupleLength());
+            map.remove(pair.getValue());
+            tupleMemoryManager.deleteTuple(pair.getKey());
+            iter.remove();
+        }
+        return buffer;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java
new file mode 100644
index 0000000..eece886
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java
@@ -0,0 +1,233 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.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;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.Utility;
+
+public class DeletableFrameTupleAppenderTest {
+    DeletableFrameTupleAppender appender;
+    ISerializerDeserializer[] fields = new ISerializerDeserializer[] {
+            IntegerSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE,
+    };
+    RecordDescriptor recordDescriptor = new RecordDescriptor(fields);
+    ArrayTupleBuilder builder = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
+    static final char TEST_CH = 'x';
+
+    int cap = 256;
+
+    @Before
+    public void initial() throws HyracksDataException {
+        appender = new DeletableFrameTupleAppender(recordDescriptor);
+    }
+
+    @Test
+    public void testClear() throws Exception {
+        ByteBuffer buffer = ByteBuffer.allocate(cap);
+        appender.clear(buffer);
+        assertTrue(appender.getBuffer() == buffer);
+        assertTrue(appender.getTupleCount() == 0);
+        assertTrue(appender.getContiguousFreeSpace() == cap - 4 - 4);
+    }
+
+    ByteBuffer makeAFrame(int capacity, int count, int deletedBytes) throws HyracksDataException {
+        ByteBuffer buffer = ByteBuffer.allocate(capacity);
+        int metaOffset = capacity - 4;
+        buffer.putInt(metaOffset, deletedBytes);
+        metaOffset -= 4;
+        buffer.putInt(metaOffset, count);
+        metaOffset -= 4;
+        for (int i = 0; i < count; i++, metaOffset -= 4) {
+            makeARecord(builder, i);
+            for (int x = 0; x < builder.getFieldEndOffsets().length; x++) {
+                buffer.putInt(builder.getFieldEndOffsets()[x]);
+            }
+            buffer.put(builder.getByteArray(), 0, builder.getSize());
+            assert (metaOffset > buffer.position());
+            buffer.putInt(metaOffset, buffer.position());
+
+        }
+        return buffer;
+    }
+
+    void makeARecord(ArrayTupleBuilder builder, int i) throws HyracksDataException {
+        builder.reset();
+        builder.addField(fields[0], i + 1);
+        builder.addField(fields[1], Utility.repeatString(TEST_CH, i + 1));
+    }
+
+    int assertTupleIsExpected(int i, int dataOffset) {
+        int tupleLength = 2 * 4 + 4 + 2 + i + 1;
+        assertEquals(dataOffset, appender.getTupleStartOffset(i));
+        assertEquals(tupleLength, appender.getTupleLength(i));
+
+        assertEquals(dataOffset + 2 * 4, appender.getAbsoluteFieldStartOffset(i, 0));
+        assertEquals(4, appender.getFieldLength(i, 0));
+        assertEquals(i + 1,
+                IntSerDeUtils.getInt(appender.getBuffer().array(), appender.getAbsoluteFieldStartOffset(i, 0)));
+        assertEquals(dataOffset + 2 * 4 + 4, appender.getAbsoluteFieldStartOffset(i, 1));
+        assertEquals(2 + i + 1, appender.getFieldLength(i, 1));
+        return tupleLength;
+    }
+
+    @Test
+    public void testReset() throws Exception {
+        ByteBuffer buffer = ByteBuffer.allocate(cap);
+        appender.reset(buffer);
+        assertTrue(appender.getBuffer() == buffer);
+        assertTrue(appender.getTupleCount() == 0);
+        assertTrue(appender.getContiguousFreeSpace() == cap - 4 - 4);
+
+        int count = 10;
+        int deleted = 7;
+        buffer = makeAFrame(cap, count, deleted);
+        int pos = buffer.position();
+        appender.reset(buffer);
+        assertTrue(appender.getBuffer() == buffer);
+        assertTrue(appender.getTupleCount() == count);
+        assertTrue(appender.getContiguousFreeSpace() == cap - 4 - 4 - count * 4 - pos);
+        assertTrue(appender.getTotalFreeSpace() == appender.getContiguousFreeSpace() + deleted);
+
+        int dataOffset = 0;
+        for (int i = 0; i < count; i++) {
+            dataOffset += assertTupleIsExpected(i, dataOffset);
+        }
+    }
+
+    @Test
+    public void testAppend() throws Exception {
+        int count = 10;
+        ByteBuffer bufferRead = makeAFrame(cap, count, 0);
+        DeletableFrameTupleAppender accessor = new DeletableFrameTupleAppender(recordDescriptor);
+        accessor.reset(bufferRead);
+        ByteBuffer bufferWrite = ByteBuffer.allocate(cap);
+        appender.clear(bufferWrite);
+        for (int i = 0; i < accessor.getTupleCount(); i++) {
+            appender.append(accessor, i);
+        }
+        for (int i = 0; i < bufferRead.capacity(); i++) {
+            assertEquals(bufferRead.get(i), bufferWrite.get(i));
+        }
+    }
+
+    @Test
+    public void testDelete() throws Exception {
+        int count = 10;
+        int deleteSpace = 0;
+        ByteBuffer buffer = makeAFrame(cap, count, deleteSpace);
+        appender.reset(buffer);
+
+        int freeSpace = appender.getContiguousFreeSpace();
+        for (int i = 0; i < appender.getTupleCount(); i++) {
+            deleteSpace += assertDeleteSucceed(i, freeSpace, deleteSpace);
+            int innerOffset = deleteSpace;
+            for (int j = i + 1; j < appender.getTupleCount(); j++) {
+                innerOffset += assertTupleIsExpected(j, innerOffset);
+            }
+        }
+    }
+
+    @Test
+    public void testResetAfterDelete() throws Exception {
+        testDelete();
+        appender.reset(appender.getBuffer());
+        assertEquals(cap - appender.getTupleCount() * 4 - 4 - 4, appender.getTotalFreeSpace());
+
+    }
+
+    int assertDeleteSucceed(int i, int freeSpaceBeforeDelete, int deleteSpace) {
+        int startOffset = appender.getTupleStartOffset(i);
+        int endOffset = appender.getTupleEndOffset(i);
+        int tupleLength = appender.getTupleLength(i);
+
+        appender.delete(i);
+
+        assertEquals(startOffset, appender.getTupleStartOffset(i));
+        assertEquals(-endOffset, appender.getTupleEndOffset(i));
+        assertEquals(-tupleLength, appender.getTupleLength(i));
+        assertEquals(freeSpaceBeforeDelete, appender.getContiguousFreeSpace());
+        assertEquals(deleteSpace + tupleLength + freeSpaceBeforeDelete, appender.getTotalFreeSpace());
+        return tupleLength;
+    }
+
+    @Test
+    public void testAppendAndDelete() throws Exception {
+        int cap = 1024;
+        int count = 10;
+        int deleteSpace = 0;
+        ByteBuffer buffer = makeAFrame(cap, count, deleteSpace);
+        int dataOffset = buffer.position();
+        appender.reset(buffer);
+
+        int freeSpace = appender.getContiguousFreeSpace();
+        int[] deleteSet = new int[] { 1, 3, 5 };
+        for (int i = 0; i < deleteSet.length; i++) {
+            deleteSpace += assertDeleteSucceed(deleteSet[i], freeSpace, deleteSpace);
+        }
+
+        ByteBuffer bufferRead = makeAFrame(cap, count * 2, 0);
+        DeletableFrameTupleAppender accessor = new DeletableFrameTupleAppender(recordDescriptor);
+        accessor.reset(bufferRead);
+
+        for (int i = count; i < accessor.getTupleCount(); i++) {
+            int id = appender.append(accessor, i);
+            dataOffset += assertTupleIsExpected(i, dataOffset);
+            assertEquals(i, id);
+        }
+
+        appender.reOrganizeBuffer();
+        dataOffset = 0;
+        for (int i = 0; i < appender.getTupleCount(); i++) {
+            if (ArrayUtils.contains(deleteSet, i)) {
+                continue;
+            }
+            dataOffset += assertTupleIsExpected(i, dataOffset);
+        }
+    }
+
+    @Test
+    public void testReOrganizeBuffer() throws Exception {
+        int count = 10;
+        testDelete();
+        appender.reOrganizeBuffer();
+        ByteBuffer bufferRead = makeAFrame(cap, count, 0);
+        DeletableFrameTupleAppender accessor = new DeletableFrameTupleAppender(recordDescriptor);
+        accessor.reset(bufferRead);
+        for (int i = 0; i < accessor.getTupleCount(); i++) {
+            appender.append(accessor, i);
+        }
+        for (int i = 0; i < bufferRead.capacity(); i++) {
+            assertEquals(bufferRead.get(i), appender.getBuffer().get(i));
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstracHeapTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstracHeapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstracHeapTest.java
new file mode 100644
index 0000000..672e078
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstracHeapTest.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Random;
+
+public class AbstracHeapTest {
+    Random random = new Random(System.currentTimeMillis());
+
+    class IntFactory implements IResetableComparableFactory<Int> {
+        @Override
+        public IResetableComparable<Int> createResetableComparable() {
+            return new Int();
+        }
+    }
+
+    class Int implements IResetableComparable<Int> {
+        int i;
+
+        public Int() {
+            i = 0;
+        }
+
+        public Int(int i) {
+            this.i = i;
+        }
+
+        @Override
+        public void reset(Int other) {
+            i = other.i;
+        }
+
+        @Override
+        public int compareTo(Int o) {
+            return Integer.compare(i, o.i);
+        }
+    }
+
+    protected void assertGetMinHeapIsSorted(IMinHeap minHeap) {
+        int count = minHeap.getNumEntries();
+        Int minI = new Int();
+        Int peekI = new Int();
+        int preI = Integer.MIN_VALUE;
+        while (!minHeap.isEmpty()) {
+            count--;
+            minHeap.peekMin(peekI);
+            minHeap.getMin(minI);
+            assertTrue(peekI.compareTo(minI) == 0);
+            assertTrue(preI <= minI.i);
+            preI = minI.i;
+        }
+        assertEquals(0, count);
+    }
+
+    protected void assertGetMaxHeapIsSorted(IMaxHeap maxHeap) {
+        int count = maxHeap.getNumEntries();
+        Int maxI = new Int();
+        Int peekI = new Int();
+        int preI = Integer.MAX_VALUE;
+        while (!maxHeap.isEmpty()) {
+            count--;
+            maxHeap.peekMax(peekI);
+            maxHeap.getMax(maxI);
+            assertTrue(peekI.compareTo(maxI) == 0);
+            assertTrue(preI >= maxI.i);
+            preI = maxI.i;
+        }
+        assertEquals(0, count);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeapTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeapTest.java
new file mode 100644
index 0000000..3f2404c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeapTest.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+public class MaxHeapTest extends AbstracHeapTest {
+    @Test
+    public void testInitialMinHeap() {
+        int capacity = 10;
+        MaxHeap maxHeap = new MaxHeap(new IntFactory(), capacity);
+        assertTrue(maxHeap.isEmpty());
+        assertEquals(0, maxHeap.getNumEntries());
+    }
+
+    @Test
+    public void testInsertSmallAmountElements() {
+        int capacity = 10;
+        MaxHeap maxHeap = new MaxHeap(new IntFactory(), capacity);
+        for (int i = 0; i < capacity; i++) {
+            maxHeap.insert(new Int(capacity - i));
+        }
+        assertEquals(capacity, maxHeap.getNumEntries());
+        assertFalse(maxHeap.isEmpty());
+
+        assertGetMaxHeapIsSorted(maxHeap);
+
+        for (int i = 0; i < capacity; i++) {
+            maxHeap.insert(new Int(random.nextInt()));
+        }
+        assertEquals(capacity, maxHeap.getNumEntries());
+        assertFalse(maxHeap.isEmpty());
+        assertGetMaxHeapIsSorted(maxHeap);
+    }
+
+    @Test
+    public void testInsertLargerThanCapacityElements() {
+        int capacity = 10;
+        MaxHeap maxHeap = new MaxHeap(new IntFactory(), capacity);
+        for (int i = 0; i < capacity; i++) {
+            maxHeap.insert(new Int(capacity - i));
+        }
+        assertEquals(capacity, maxHeap.getNumEntries());
+        assertFalse(maxHeap.isEmpty());
+        assertGetMaxHeapIsSorted(maxHeap);
+
+        for (int i = 0; i < capacity * 10; i++) {
+            maxHeap.insert(new Int(random.nextInt()));
+        }
+        assertEquals(capacity * 10, maxHeap.getNumEntries());
+        assertFalse(maxHeap.isEmpty());
+        assertGetMaxHeapIsSorted(maxHeap);
+
+    }
+
+    @Test
+    public void testReplaceMax() {
+        int capacity = 10;
+        MaxHeap maxHeap = new MaxHeap(new IntFactory(), capacity);
+        for (int i = capacity; i < capacity * 2; i++) {
+            maxHeap.insert(new Int(i));
+        }
+        assertEquals(capacity, maxHeap.getNumEntries());
+        assertFalse(maxHeap.isEmpty());
+
+        for (int i = 0; i < capacity; i++) {
+            maxHeap.replaceMax(new Int(i));
+        }
+        assertEquals(capacity, maxHeap.getNumEntries());
+        assertFalse(maxHeap.isEmpty());
+
+        Int maxI = new Int();
+        Int peekI = new Int();
+        int i = 0;
+        while (!maxHeap.isEmpty()) {
+            maxHeap.peekMax(peekI);
+            maxHeap.getMax(maxI);
+            assertTrue(peekI.compareTo(maxI) == 0);
+            assertEquals(  i++, capacity - 1 - maxI.i);
+        }
+    }
+}


[14/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
VariableSizeFrame(VSizeFrame) support for Hyracks.

This patch replaced Frame/Accessor/Appender with the new API which
supports BigObject.
The ExternalSorter/TopKSorter/ExternalGroupSorter
have been implemented to support big object.

The Groupby && Join should work with BigObject also. But it will break the
memory budget when it encounter a big object. I will fix the memory
problem later in a separate CR.

The design about the frame allocation is
here:https://docs.google.com/presentation/d/15h9iQf5OYsgGZoQTbGHkj1yS2G9q2fd0s1lDAD1EJq0/edit?usp=sharing

Suggest review order:
Patch 12: It includes all of the sorting operators.
Patch 13: It applys the new IFrame API to all Hyracks codes.
Patch 14: Some bug fixes to pass all Asterix's tests.
Patch 15: Skip it!
Patch 16: Some bug fixes to the Asterix's tests in small frame setting.
Later Patch: address the comments

Change-Id: I2e08692078683f6f2cf17387e39037ad851fc05b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/234
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/commit/0d87a57f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/tree/0d87a57f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/diff/0d87a57f

Branch: refs/heads/master
Commit: 0d87a57f7439eca69e6dae73f117747b4ea51746
Parents: 0e5d531
Author: JavierJia <ji...@gmail.com>
Authored: Wed Jun 17 20:07:38 2015 -0700
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Wed Jun 17 20:45:14 2015 -0700

----------------------------------------------------------------------
 ...estedPlansAccumulatingAggregatorFactory.java |   6 +-
 .../NestedPlansRunningAggregatorFactory.java    |  27 +-
 ...eInputOneOutputOneFieldFramePushRuntime.java |  41 ++
 ...actOneInputOneOutputOneFramePushRuntime.java |  96 +--
 ...AbstractOneInputOneOutputRuntimeFactory.java |   2 +-
 .../MicroPreClusteredGroupRuntimeFactory.java   |  61 +-
 .../operators/meta/SubplanRuntimeFactory.java   |  14 +-
 .../sort/InMemorySortRuntimeFactory.java        |  13 +-
 .../std/EmptyTupleSourceRuntimeFactory.java     |  10 +-
 .../std/NestedTupleSourceRuntimeFactory.java    |   4 +-
 .../PartitioningSplitOperatorDescriptor.java    |  28 +-
 .../operators/std/SinkWriterRuntime.java        |   4 +-
 .../std/StreamSelectRuntimeFactory.java         |  49 +-
 .../ics/hyracks/api/comm/FixedSizeFrame.java    |  59 ++
 .../ics/hyracks/api/comm/FrameConstants.java    |  31 +-
 .../uci/ics/hyracks/api/comm/FrameHelper.java   |  61 +-
 .../edu/uci/ics/hyracks/api/comm/IFrame.java    |  60 ++
 .../ics/hyracks/api/comm/IFrameAppender.java    |  55 ++
 .../hyracks/api/comm/IFrameFieldAppender.java   |  47 ++
 .../uci/ics/hyracks/api/comm/IFrameReader.java  |   8 +-
 .../hyracks/api/comm/IFrameTupleAccessor.java   |  25 +-
 .../hyracks/api/comm/IFrameTupleAppender.java   |  39 +
 .../uci/ics/hyracks/api/comm/IFrameWriter.java  |  11 +-
 .../hyracks/api/comm/NoShrinkVSizeFrame.java    |  31 +
 .../uci/ics/hyracks/api/comm/VSizeFrame.java    |  76 ++
 .../api/context/IHyracksCommonContext.java      |  10 +-
 .../api/context/IHyracksFrameMgrContext.java    |  39 +
 .../api/dataset/IHyracksDatasetReader.java      |   5 +-
 .../client/dataset/DatasetClientContext.java    |  21 +-
 .../client/dataset/HyracksDatasetReader.java    |  27 +-
 .../comm/channels/NetworkInputChannel.java      |   2 +-
 .../comm/channels/NetworkOutputChannel.java     |  52 +-
 .../comm/channels/ReadBufferFactory.java        |   2 +-
 .../edu/uci/ics/hyracks/control/nc/Joblet.java  |  35 +-
 .../edu/uci/ics/hyracks/control/nc/Task.java    |  24 +-
 .../nc/dataset/DatasetPartitionWriter.java      |   2 +-
 .../hyracks/control/nc/dataset/ResultState.java |   3 +
 .../ics/hyracks/control/nc/io/IOManager.java    |  11 +-
 .../MaterializedPartitionInputChannel.java      |   3 +
 .../control/nc/partitions/PartitionManager.java |   4 +-
 .../ConnectorReceiverProfilingFrameReader.java  |   7 +-
 .../nc/resources/memory/FrameManager.java       |  85 +++
 hyracks/hyracks-dataflow-common/pom.xml         |   6 +
 .../common/comm/io/AbstractFrameAppender.java   |  99 +++
 .../common/comm/io/ArrayTupleBuilder.java       |   1 +
 .../dataflow/common/comm/io/FrameConstants.java |  23 -
 .../common/comm/io/FrameDeserializer.java       |   5 +-
 .../comm/io/FrameDeserializingDataReader.java   |  19 +-
 .../comm/io/FrameDeserializingDataWriter.java   |   2 +-
 .../common/comm/io/FrameFixedFieldAppender.java | 108 +++
 .../comm/io/FrameFixedFieldTupleAppender.java   | 130 ++++
 .../common/comm/io/FrameOutputStream.java       |  24 +-
 .../common/comm/io/FrameTupleAccessor.java      | 101 ++-
 .../common/comm/io/FrameTupleAppender.java      | 237 +++---
 .../comm/io/FrameTupleAppenderAccessor.java     | 131 ++++
 .../comm/io/FrameTupleAppenderWrapper.java      |  80 +--
 .../comm/io/ResultFrameTupleAccessor.java       |  72 +-
 .../common/comm/io/SerializingDataWriter.java   |  32 +-
 .../dataflow/common/comm/util/FrameUtils.java   | 260 ++++++-
 .../dataflow/common/io/RunFileReader.java       |  28 +-
 .../dataflow/common/util/IntSerDeUtils.java     |   6 +
 .../FrameFixedFieldTupleAppenderTest.java       | 215 ++++++
 .../hadoop/HadoopReadOperatorDescriptor.java    |  21 +-
 .../dataflow/hadoop/mapreduce/HadoopHelper.java |   2 +-
 .../dataflow/hadoop/mapreduce/KVIterator.java   |  20 +-
 .../mapreduce/MapperOperatorDescriptor.java     |  50 +-
 .../dataflow/hadoop/mapreduce/ReduceWriter.java |  32 +-
 .../hadoop/mapreduce/ShuffleFrameReader.java    |  83 ++-
 hyracks/hyracks-dataflow-std/pom.xml            |   6 +
 .../std/collectors/InputChannelFrameReader.java |  69 +-
 .../NonDeterministicChannelReader.java          |  27 +-
 .../collectors/NonDeterministicFrameReader.java |  30 +-
 .../std/collectors/SortMergeFrameReader.java    |  17 +-
 .../LocalityAwarePartitionDataWriter.java       |  34 +-
 .../std/connectors/PartitionDataWriter.java     |  42 +-
 .../file/DelimitedDataTupleParserFactory.java   |  26 +-
 .../file/PlainFileWriterOperatorDescriptor.java |   9 +-
 .../std/group/HashSpillableTableFactory.java    |  63 +-
 .../dataflow/std/group/ISpillableTable.java     |   4 +-
 .../ExternalGroupBuildOperatorNodePushable.java |   2 +-
 .../ExternalGroupMergeOperatorNodePushable.java |  63 +-
 .../std/group/hash/GroupingHashTable.java       |  45 +-
 .../HashGroupBuildOperatorNodePushable.java     |   2 +-
 .../PreclusteredGroupOperatorNodePushable.java  |   8 -
 .../preclustered/PreclusteredGroupWriter.java   |  29 +-
 .../sort/ExternalSortGroupByRunGenerator.java   |  98 +--
 .../sort/ExternalSortGroupByRunMerger.java      | 144 +---
 .../sort/SortGroupByOperatorDescriptor.java     | 255 ++-----
 .../join/GraceHashJoinOperatorNodePushable.java |  20 +-
 ...hJoinPartitionBuildOperatorNodePushable.java |  20 +-
 .../join/HybridHashJoinOperatorDescriptor.java  | 106 +--
 .../dataflow/std/join/InMemoryHashJoin.java     |  52 +-
 .../InMemoryHashJoinOperatorDescriptor.java     |  26 +-
 .../dataflow/std/join/NestedLoopJoin.java       |  96 +--
 .../join/NestedLoopJoinOperatorDescriptor.java  |  22 +-
 .../std/join/OptimizedHybridHashJoin.java       | 122 ++--
 ...timizedHybridHashJoinOperatorDescriptor.java | 130 ++--
 ...ConstantTupleSourceOperatorNodePushable.java |  10 +-
 .../std/misc/LimitOperatorDescriptor.java       |  12 +-
 .../std/misc/MaterializerTaskState.java         |   8 +-
 .../misc/MaterializingOperatorDescriptor.java   |   9 +-
 .../std/misc/SplitOperatorDescriptor.java       |   4 +-
 .../result/ResultWriterOperatorDescriptor.java  |  19 +-
 .../dataflow/std/sort/AbstractFrameSorter.java  | 186 +++++
 .../std/sort/AbstractSortRunGenerator.java      |  77 ++
 .../sort/AbstractSorterOperatorDescriptor.java  | 197 +++++
 .../hyracks/dataflow/std/sort/BSTMemMgr.java    | 717 -------------------
 .../hyracks/dataflow/std/sort/BSTNodeUtil.java  | 233 ------
 .../sort/ExternalSortOperatorDescriptor.java    | 211 ++----
 .../std/sort/ExternalSortRunGenerator.java      | 127 ++--
 .../std/sort/ExternalSortRunMerger.java         | 435 +++++------
 .../dataflow/std/sort/FrameSorterMergeSort.java | 172 +----
 .../dataflow/std/sort/FrameSorterQuickSort.java | 157 +---
 .../dataflow/std/sort/HeapSortRunGenerator.java |  99 +++
 .../std/sort/HybridTopKSortRunGenerator.java    | 109 +++
 .../hyracks/dataflow/std/sort/IFrameSorter.java |  31 +-
 .../dataflow/std/sort/IMemoryManager.java       |  88 ---
 .../dataflow/std/sort/IRunGenerator.java        |   3 +-
 .../dataflow/std/sort/ISelectionTree.java       |  90 ---
 .../ics/hyracks/dataflow/std/sort/ISorter.java  |  33 +
 .../hyracks/dataflow/std/sort/ITupleSorter.java |  26 +
 .../sort/InMemorySortOperatorDescriptor.java    |  27 +-
 ...OptimizedExternalSortOperatorDescriptor.java | 218 ------
 .../sort/OptimizedExternalSortRunGenerator.java | 283 --------
 ...imizedExternalSortRunGeneratorWithLimit.java | 436 -----------
 .../std/sort/RunAndMaxFrameSizePair.java        |  32 +
 .../std/sort/RunMergingFrameReader.java         | 115 +--
 .../uci/ics/hyracks/dataflow/std/sort/Slot.java |  81 ---
 .../hyracks/dataflow/std/sort/SortMinHeap.java  | 293 --------
 .../dataflow/std/sort/SortMinMaxHeap.java       | 448 ------------
 .../std/sort/TopKSorterOperatorDescriptor.java  |  62 ++
 .../dataflow/std/sort/TupleSorterHeapSort.java  | 269 +++++++
 .../sort/buffermanager/EnumFreeSlotPolicy.java  |  22 +
 .../FrameFreeSlotBiggestFirst.java              |  97 +++
 .../buffermanager/FrameFreeSlotLastFit.java     |  81 +++
 .../buffermanager/FrameFreeSlotSmallestFit.java |  59 ++
 .../sort/buffermanager/IFrameBufferManager.java |  68 ++
 .../buffermanager/IFrameFreeSlotPolicy.java     |  44 ++
 .../std/sort/buffermanager/IFramePool.java      |  48 ++
 .../buffermanager/ITupleBufferAccessor.java     |  36 +
 .../sort/buffermanager/ITupleBufferManager.java |  42 ++
 .../VariableFrameMemoryManager.java             | 132 ++++
 .../sort/buffermanager/VariableFramePool.java   | 200 ++++++
 .../VariableTupleMemoryManager.java             | 203 ++++++
 .../sort/util/DeletableFrameTupleAppender.java  | 244 +++++++
 .../std/sort/util/GroupFrameAccessor.java       | 170 +++++
 .../dataflow/std/sort/util/GroupVSizeFrame.java |  46 ++
 .../IAppendDeletableFrameTupleAccessor.java     |  72 ++
 .../dataflow/std/structures/AbstractHeap.java   | 156 ++++
 .../hyracks/dataflow/std/structures/IHeap.java  |  44 ++
 .../dataflow/std/structures/IMaxHeap.java       |  43 ++
 .../dataflow/std/structures/IMinHeap.java       |  42 ++
 .../dataflow/std/structures/IMinMaxHeap.java    |  18 +
 .../dataflow/std/structures/IResetable.java     |  20 +
 .../std/structures/IResetableComparable.java    |  19 +
 .../structures/IResetableComparableFactory.java |  20 +
 .../dataflow/std/structures/MaxHeap.java        |  63 ++
 .../dataflow/std/structures/MinHeap.java        |  62 ++
 .../dataflow/std/structures/MinMaxHeap.java     | 217 ++++++
 .../std/structures/SerializableHashTable.java   |   2 +-
 .../dataflow/std/structures/TuplePointer.java   |  43 +-
 .../util/DeserializedOperatorNodePushable.java  |   2 +-
 .../ics/hyracks/dataflow/std/util/MathUtil.java |  50 ++
 .../dataflow/std/util/ReferenceEntry.java       |  14 +-
 .../std/util/ReferencedPriorityQueue.java       |  31 +-
 .../ics/hyracks/dataflow/std/sort/Utility.java  |  23 +
 .../dataflow/std/sort/buffermanager/Common.java |  26 +
 .../FrameFreeSlotBestFitUsingTreeMapTest.java   |  60 ++
 .../FrameFreeSlotBiggestFirstTest.java          |  70 ++
 .../buffermanager/FrameFreeSlotLastFitTest.java |  86 +++
 .../buffermanager/VariableFramePoolTest.java    | 216 ++++++
 .../VariableFramesMemoryManagerTest.java        | 170 +++++
 .../VariableTupleMemoryManagerTest.java         | 230 ++++++
 .../util/DeletableFrameTupleAppenderTest.java   | 233 ++++++
 .../std/structures/AbstracHeapTest.java         |  86 +++
 .../dataflow/std/structures/MaxHeapTest.java    |  99 +++
 .../dataflow/std/structures/MinHeapTest.java    | 102 +++
 .../dataflow/std/structures/MinMaxHeapTest.java | 109 +++
 .../ics/hyracks/dataflow/std/util/MathTest.java |  40 ++
 .../btree/helper/DataGenOperatorDescriptor.java |  12 +-
 .../hyracks-integration-tests/.gitignore        |   3 +
 .../comm/SerializationDeserializationTest.java  |  20 +-
 .../integration/AbstractIntegrationTest.java    |  37 +-
 .../AbstractMultiNCIntegrationTest.java         |  21 +-
 .../integration/OptimizedSortMergeTest.java     |  31 +-
 .../integration/VSizeFrameSortMergeTest.java    | 118 +++
 .../tests/unit/AbstractRunGeneratorTest.java    | 279 ++++++++
 .../unit/ExternalSortRunGeneratorTest.java      |  32 +
 .../tests/unit/HeapSortRunGeneratorTest.java    |  37 +
 .../tests/unit/HybridSortRunGenerator.java      |  30 +
 .../tests/unit/RunMergingFrameReaderTest.java   | 409 +++++++++++
 .../tests/unit/TopKRunGeneratorTest.java        | 208 ++++++
 .../examples/text/WordTupleParserFactory.java   |  19 +-
 .../tpch-example/tpchclient/pom.xml             |   2 +-
 .../hyracks/examples/tpch/client/Common.java    |  83 +++
 .../ics/hyracks/examples/tpch/client/Join.java  | 320 +++++++++
 .../ics/hyracks/examples/tpch/client/Main.java  | 362 ----------
 .../ics/hyracks/examples/tpch/client/Sort.java  | 165 +++++
 .../dataflow/HDFSWriteOperatorDescriptor.java   |   2 +-
 .../hdfs/lib/TextKeyValueParserFactory.java     |  18 +-
 .../dataflow/HDFSWriteOperatorDescriptor.java   |   2 +-
 .../BTreeUpdateSearchOperatorNodePushable.java  |  11 +-
 .../IndexBulkLoadOperatorNodePushable.java      |   2 +-
 ...xInsertUpdateDeleteOperatorNodePushable.java |  13 +-
 .../IndexSearchOperatorNodePushable.java        |  34 +-
 ...eIndexDiskOrderScanOperatorNodePushable.java |  22 +-
 .../TreeIndexStatsOperatorNodePushable.java     |  13 +-
 ...xInsertUpdateDeleteOperatorNodePushable.java |  27 +-
 .../BinaryTokenizerOperatorNodePushable.java    |  31 +-
 ...InvertedIndexSearchOperatorNodePushable.java |   2 +
 .../ondisk/FixedSizeFrameTupleAccessor.java     |  10 +
 .../ondisk/OnDiskInvertedIndex.java             |  12 +-
 .../search/AbstractTOccurrenceSearcher.java     |  26 +-
 .../search/PartitionedTOccurrenceSearcher.java  |   9 +-
 .../lsm/invertedindex/search/SearchResult.java  |   4 +-
 .../search/TOccurrenceSearcher.java             |   4 +-
 .../hyracks/test/support/TestJobletContext.java |  20 +-
 .../hyracks/test/support/TestTaskContext.java   |  21 +-
 .../storage/am/btree/BTreeStatsTest.java        |  11 +-
 .../storage/am/btree/FieldPrefixNSMTest.java    |  11 +-
 220 files changed, 10258 insertions(+), 6056 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
index af4bff2..618768c 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansAccumulatingAggregatorFactory.java
@@ -50,7 +50,7 @@ public class NestedPlansAccumulatingAggregatorFactory extends AbstractAccumulati
     public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
             RecordDescriptor outRecordDescriptor, int[] keys, int[] partialKeys) throws HyracksDataException {
 
-        final AggregatorOutput outputWriter = new AggregatorOutput(ctx.getFrameSize(), subplans, keyFieldIdx.length,
+        final AggregatorOutput outputWriter = new AggregatorOutput(subplans, keyFieldIdx.length,
                 decorFieldIdx.length);
         final NestedTupleSourceRuntime[] pipelines = new NestedTupleSourceRuntime[subplans.length];
         for (int i = 0; i < subplans.length; i++) {
@@ -173,7 +173,7 @@ public class NestedPlansAccumulatingAggregatorFactory extends AbstractAccumulati
         private ArrayTupleBuilder tb;
         private AlgebricksPipeline[] subplans;
 
-        public AggregatorOutput(int frameSize, AlgebricksPipeline[] subplans, int numKeys, int numDecors) {
+        public AggregatorOutput(AlgebricksPipeline[] subplans, int numKeys, int numDecors) {
             this.subplans = subplans;
             // this.keyFieldIndexes = keyFieldIndexes;
             int totalAggFields = 0;
@@ -187,7 +187,7 @@ public class NestedPlansAccumulatingAggregatorFactory extends AbstractAccumulati
 
             this.tAccess = new FrameTupleAccessor[inputRecDesc.length];
             for (int i = 0; i < inputRecDesc.length; i++) {
-                tAccess[i] = new FrameTupleAccessor(frameSize, inputRecDesc[i]);
+                tAccess[i] = new FrameTupleAccessor(inputRecDesc[i]);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
index c8dc852..b7e736e 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java
@@ -23,6 +23,7 @@ import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.std.NestedTupleSourceRuntimeFactory.NestedTupleSourceRuntime;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 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;
@@ -67,10 +68,6 @@ public class NestedPlansRunningAggregatorFactory implements IAggregatorDescripto
 
         final ArrayTupleBuilder gbyTb = outputWriter.getGroupByTupleBuilder();
 
-        final ByteBuffer outputFrame = ctx.allocateFrame();
-        final FrameTupleAppender outputAppender = new FrameTupleAppender(ctx.getFrameSize());
-        outputAppender.reset(outputFrame, true);
-
         return new IAggregatorDescriptor() {
 
             @Override
@@ -167,7 +164,6 @@ public class NestedPlansRunningAggregatorFactory implements IAggregatorDescripto
         private final ArrayTupleBuilder gbyTb;
         private final AlgebricksPipeline[] subplans;
         private final IFrameWriter outputWriter;
-        private final ByteBuffer outputFrame;
         private final FrameTupleAppender outputAppender;
 
         public RunningAggregatorOutput(IHyracksTaskContext ctx, AlgebricksPipeline[] subplans, int numKeys,
@@ -188,12 +184,10 @@ public class NestedPlansRunningAggregatorFactory implements IAggregatorDescripto
 
             this.tAccess = new FrameTupleAccessor[inputRecDesc.length];
             for (int i = 0; i < inputRecDesc.length; i++) {
-                tAccess[i] = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc[i]);
+                tAccess[i] = new FrameTupleAccessor(inputRecDesc[i]);
             }
 
-            this.outputFrame = ctx.allocateFrame();
-            this.outputAppender = new FrameTupleAppender(ctx.getFrameSize());
-            this.outputAppender.reset(outputFrame, true);
+            this.outputAppender = new FrameTupleAppender(new VSizeFrame(ctx));
         }
 
         @Override
@@ -221,23 +215,14 @@ public class NestedPlansRunningAggregatorFactory implements IAggregatorDescripto
                 for (int f = 0; f < w; f++) {
                     tb.addField(accessor, tIndex, f);
                 }
-                if (!outputAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    FrameUtils.flushFrame(outputFrame, outputWriter);
-                    outputAppender.reset(outputFrame, true);
-                    if (!outputAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                        throw new HyracksDataException(
-                                "Failed to write a running aggregation result into an empty frame: possibly the size of the result is too large.");
-                    }
-                }
+                FrameUtils.appendToWriter(outputWriter, outputAppender, tb.getFieldEndOffsets(),
+                        tb.getByteArray(), 0, tb.getSize());
             }
         }
 
         @Override
         public void close() throws HyracksDataException {
-            if (outputAppender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(outputFrame, outputWriter);
-                outputAppender.reset(outputFrame, true);
-            }
+            outputAppender.flush(outputWriter, true);
         }
 
         public void setInputIdx(int inputIdx) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFieldFramePushRuntime.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFieldFramePushRuntime.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFieldFramePushRuntime.java
new file mode 100644
index 0000000..44c1736
--- /dev/null
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFieldFramePushRuntime.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hyracks.algebricks.runtime.operators.base;
+
+import edu.uci.ics.hyracks.api.comm.IFrameFieldAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameFixedFieldTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+
+public abstract class AbstractOneInputOneOutputOneFieldFramePushRuntime
+        extends AbstractOneInputOneOutputOneFramePushRuntime {
+
+    @Override
+    protected IFrameTupleAppender getTupleAppender() {
+        return (FrameFixedFieldTupleAppender) appender;
+    }
+
+    protected IFrameFieldAppender getFieldAppender() {
+        return (FrameFixedFieldTupleAppender) appender;
+    }
+
+    protected final void initAccessAppendFieldRef(IHyracksTaskContext ctx) throws HyracksDataException {
+        frame = new VSizeFrame(ctx);
+        appender = new FrameFixedFieldTupleAppender(inputRecordDesc.getFieldCount());
+        appender.reset(frame, true);
+        tAccess = new FrameTupleAccessor(inputRecordDesc);
+        tRef = new FrameTupleReference();
+    }
+
+    protected void appendField(byte[] array, int start, int length) throws HyracksDataException {
+        FrameUtils.appendFieldToWriter(writer, getFieldAppender(), array, start, length);
+    }
+
+    protected void appendField(IFrameTupleAccessor accessor, int tid, int fid) throws HyracksDataException {
+        FrameUtils.appendFieldToWriter(writer, getFieldAppender(), accessor, tid, fid);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
index 81052d6..ec4e039 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputOneFramePushRuntime.java
@@ -14,8 +14,11 @@
  */
 package edu.uci.ics.hyracks.algebricks.runtime.operators.base;
 
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -26,42 +29,53 @@ import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 
 public abstract class AbstractOneInputOneOutputOneFramePushRuntime extends AbstractOneInputOneOutputPushRuntime {
 
-    protected FrameTupleAppender appender;
-    protected ByteBuffer frame;
+    protected IFrameAppender appender;
+    protected IFrame frame;
     protected FrameTupleAccessor tAccess;
     protected FrameTupleReference tRef;
 
+    protected final void initAccessAppend(IHyracksTaskContext ctx) throws HyracksDataException {
+        frame = new VSizeFrame(ctx);
+        appender = new FrameTupleAppender(frame);
+        tAccess = new FrameTupleAccessor(inputRecordDesc);
+    }
+
+    protected final void initAccessAppendRef(IHyracksTaskContext ctx) throws HyracksDataException {
+        initAccessAppend(ctx);
+        tRef = new FrameTupleReference();
+    }
+
     @Override
     public void close() throws HyracksDataException {
         flushIfNotFailed();
         writer.close();
-        appender.reset(frame, true);
+    }
+
+    protected void flushAndReset() throws HyracksDataException {
+        if (appender.getTupleCount() > 0) {
+            appender.flush(writer, true);
+        }
     }
 
     protected void flushIfNotFailed() throws HyracksDataException {
         if (!failed) {
-            if (appender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(frame, writer);
-            }
+            flushAndReset();
         }
     }
 
+    protected IFrameTupleAppender getTupleAppender() {
+        return (FrameTupleAppender) appender;
+    }
+
     protected void appendToFrameFromTupleBuilder(ArrayTupleBuilder tb) throws HyracksDataException {
         appendToFrameFromTupleBuilder(tb, false);
     }
 
     protected void appendToFrameFromTupleBuilder(ArrayTupleBuilder tb, boolean flushFrame) throws HyracksDataException {
-        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
-            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new HyracksDataException(
-                        "Could not write frame: the size of the tuple is too long to be fit into a single frame. (AbstractOneInputOneOutputOneFramePushRuntime.appendToFrameFromTupleBuilder)");
-            }
-        }
+        FrameUtils.appendToWriter(writer, getTupleAppender(), tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+                tb.getSize());
         if (flushFrame) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            flushAndReset();
         }
     }
 
@@ -71,52 +85,18 @@ public abstract class AbstractOneInputOneOutputOneFramePushRuntime extends Abstr
 
     protected void appendProjectionToFrame(int tIndex, int[] projectionList, boolean flushFrame)
             throws HyracksDataException {
-        if (!appender.appendProjection(tAccess, tIndex, projectionList)) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
-            if (!appender.appendProjection(tAccess, tIndex, projectionList)) {
-                throw new IllegalStateException(
-                        "Could not write frame (AbstractOneInputOneOutputOneFramePushRuntime.appendProjectionToFrame).");
-            }
-            return;
-        }
+        FrameUtils.appendProjectionToWriter(writer, getTupleAppender(), tAccess, tIndex, projectionList);
         if (flushFrame) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            flushAndReset();
         }
     }
 
     protected void appendTupleToFrame(int tIndex) throws HyracksDataException {
-        if (!appender.append(tAccess, tIndex)) {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
-            if (!appender.append(tAccess, tIndex)) {
-                throw new IllegalStateException(
-                        "Could not write frame (AbstractOneInputOneOutputOneFramePushRuntime.appendTupleToFrame).");
-            }
-        }
-    }
-
-    protected final void initAccessAppend(IHyracksTaskContext ctx) throws HyracksDataException {
-        // if (allocFrame) {
-        frame = ctx.allocateFrame();
-        appender = new FrameTupleAppender(ctx.getFrameSize());
-        appender.reset(frame, true);
-        // }
-        tAccess = new FrameTupleAccessor(ctx.getFrameSize(), inputRecordDesc);
-    }
-
-    protected final void initAccessAppendRef(IHyracksTaskContext ctx) throws HyracksDataException {
-        initAccessAppend(ctx);
-        tRef = new FrameTupleReference();
+        FrameUtils.appendToWriter(writer, getTupleAppender(), tAccess, tIndex);
     }
 
-    protected final void initAccessAppendFieldRef(IHyracksTaskContext ctx) throws HyracksDataException {
-        frame = ctx.allocateFrame();
-        appender = new FrameTupleAppender(ctx.getFrameSize(), inputRecordDesc.getFieldCount());
-        appender.reset(frame, true);
-        tAccess = new FrameTupleAccessor(ctx.getFrameSize(), inputRecordDesc);
-        tRef = new FrameTupleReference();
+    protected void appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
+            throws HyracksDataException {
+        FrameUtils.appendConcatToWriter(writer, getTupleAppender(), accessor0, tIndex0, accessor1, tIndex1);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
index 43270b6..1d31916 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/base/AbstractOneInputOneOutputRuntimeFactory.java
@@ -24,7 +24,7 @@ public abstract class AbstractOneInputOneOutputRuntimeFactory implements IPushRu
 
     private static final long serialVersionUID = 1L;
 
-    protected int[] projectionList;
+    protected final int[] projectionList;
 
     public AbstractOneInputOneOutputRuntimeFactory(int[] projectionList) {
         this.projectionList = projectionList;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
index 2e1171c..3380d08 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/group/MicroPreClusteredGroupRuntimeFactory.java
@@ -26,7 +26,6 @@ import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupWriter;
 
@@ -57,47 +56,37 @@ public class MicroPreClusteredGroupRuntimeFactory extends AbstractOneInputOneOut
     @Override
     public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx)
             throws AlgebricksException {
-        try {
-            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
-            for (int i = 0; i < comparatorFactories.length; ++i) {
-                comparators[i] = comparatorFactories[i].createBinaryComparator();
-            }
-            final ByteBuffer copyFrame = ctx.allocateFrame();
-            final FrameTupleAccessor copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
-            copyFrameAccessor.reset(copyFrame);
-            ByteBuffer outFrame = ctx.allocateFrame();
-            final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-            appender.reset(outFrame, true);
+        final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
 
-            return new AbstractOneInputOneOutputPushRuntime() {
+        return new AbstractOneInputOneOutputPushRuntime() {
 
-                private PreclusteredGroupWriter pgw;
+            private PreclusteredGroupWriter pgw;
 
-                @Override
-                public void open() throws HyracksDataException {
-                    pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregatorFactory, inRecordDesc,
-                            outRecordDesc, writer);
-                    pgw.open();
-                }
+            @Override
+            public void open() throws HyracksDataException {
+                pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregatorFactory, inRecordDesc,
+                        outRecordDesc, writer);
+                pgw.open();
+            }
 
-                @Override
-                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    pgw.nextFrame(buffer);
-                }
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                pgw.nextFrame(buffer);
+            }
 
-                @Override
-                public void fail() throws HyracksDataException {
-                    pgw.fail();
-                }
+            @Override
+            public void fail() throws HyracksDataException {
+                pgw.fail();
+            }
 
-                @Override
-                public void close() throws HyracksDataException {
-                    pgw.close();
-                }
-            };
-        } catch (HyracksDataException e) {
-            throw new AlgebricksException(e);
-        }
+            @Override
+            public void close() throws HyracksDataException {
+                pgw.close();
+            }
+        };
 
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
index e5bede2..cf40669 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -33,7 +33,6 @@ 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.util.FrameUtils;
 
 public class SubplanRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
@@ -87,9 +86,7 @@ public class SubplanRuntimeFactory extends AbstractOneInputOneOutputRuntimeFacto
             class TupleOuterProduct implements IFrameWriter {
 
                 private boolean smthWasWritten = false;
-                private IHyracksTaskContext hCtx = ctx;
-                private int frameSize = hCtx.getFrameSize();
-                private FrameTupleAccessor ta = new FrameTupleAccessor(frameSize,
+                private FrameTupleAccessor ta = new FrameTupleAccessor(
                         pipeline.getRecordDescriptors()[pipeline.getRecordDescriptors().length - 1]);
                 private ArrayTupleBuilder tb = new ArrayTupleBuilder(nullWriters.length);
 
@@ -103,14 +100,7 @@ public class SubplanRuntimeFactory extends AbstractOneInputOneOutputRuntimeFacto
                     ta.reset(buffer);
                     int nTuple = ta.getTupleCount();
                     for (int t = 0; t < nTuple; t++) {
-                        if (!appender.appendConcat(tRef.getFrameTupleAccessor(), tRef.getTupleIndex(), ta, t)) {
-                            FrameUtils.flushFrame(frame, writer);
-                            appender.reset(frame, true);
-                            if (!appender.appendConcat(tRef.getFrameTupleAccessor(), tRef.getTupleIndex(), ta, t)) {
-                                throw new HyracksDataException(
-                                        "Could not write frame: subplan result is larger than the single-frame limit.");
-                            }
-                        }
+                        appendConcat(tRef.getFrameTupleAccessor(), tRef.getTupleIndex(), ta, t);
                     }
                     smthWasWritten = true;
                 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
index 54ed192..d3751f5 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
@@ -25,6 +25,10 @@ import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorterMergeSort;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotLastFit;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
 
 public class InMemorySortRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
@@ -57,7 +61,10 @@ public class InMemorySortRuntimeFactory extends AbstractOneInputOneOutputRuntime
             @Override
             public void open() throws HyracksDataException {
                 if (frameSorter == null) {
-                    frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory,
+                    IFrameBufferManager manager = new VariableFrameMemoryManager(
+                            new VariableFramePool(ctx, VariableFramePool.UNLIMITED_MEMORY),
+                            new FrameFreeSlotLastFit());
+                    frameSorter = new FrameSorterMergeSort(ctx, manager, sortFields, firstKeyNormalizerFactory,
                             comparatorFactories, outputRecordDesc);
                 }
                 frameSorter.reset();
@@ -76,8 +83,8 @@ public class InMemorySortRuntimeFactory extends AbstractOneInputOneOutputRuntime
 
             @Override
             public void close() throws HyracksDataException {
-                frameSorter.sortFrames();
-                frameSorter.flushFrames(writer);
+                frameSorter.sort();
+                frameSorter.flush(writer);
                 writer.close();
             }
         };

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
index 745fdf6..35fcafc 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/EmptyTupleSourceRuntimeFactory.java
@@ -14,16 +14,14 @@
  */
 package edu.uci.ics.hyracks.algebricks.runtime.operators.std;
 
-import java.nio.ByteBuffer;
-
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntime;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputSourcePushRuntime;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 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.comm.util.FrameUtils;
 
 public class EmptyTupleSourceRuntimeFactory implements IPushRuntimeFactory {
 
@@ -41,18 +39,16 @@ public class EmptyTupleSourceRuntimeFactory implements IPushRuntimeFactory {
     public IPushRuntime createPushRuntime(final IHyracksTaskContext ctx) throws HyracksDataException {
         return new AbstractOneInputSourcePushRuntime() {
 
-            private ByteBuffer frame = ctx.allocateFrame();
             private ArrayTupleBuilder tb = new ArrayTupleBuilder(0);
-            private FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+            private FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
 
             @Override
             public void open() throws HyracksDataException {
                 writer.open();
-                appender.reset(frame, true);
                 if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                     throw new IllegalStateException();
                 }
-                FrameUtils.flushFrame(frame, writer);
+                appender.flush(writer, true);
                 writer.close();
             }
         };

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
index 55dbcbb..8df87ab 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/NestedTupleSourceRuntimeFactory.java
@@ -21,7 +21,6 @@ import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 public class NestedTupleSourceRuntimeFactory implements IPushRuntimeFactory {
 
@@ -67,8 +66,7 @@ public class NestedTupleSourceRuntimeFactory implements IPushRuntimeFactory {
         }
 
         public void forceFlush() throws HyracksDataException {
-            FrameUtils.flushFrame(frame, writer);
-            appender.reset(frame, true);
+            appender.flush(writer, true);
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/PartitioningSplitOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/PartitioningSplitOperatorDescriptor.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/PartitioningSplitOperatorDescriptor.java
index d19dd34..1d2d27c 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/PartitioningSplitOperatorDescriptor.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/PartitioningSplitOperatorDescriptor.java
@@ -22,7 +22,9 @@ import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -63,15 +65,15 @@ public class PartitioningSplitOperatorDescriptor extends AbstractSingleActivityO
             throws HyracksDataException {
         return new AbstractUnaryInputOperatorNodePushable() {
             private final IFrameWriter[] writers = new IFrameWriter[outputArity];
-            private final ByteBuffer[] writeBuffers = new ByteBuffer[outputArity];
+            private final IFrame[] writeBuffers = new IFrame[outputArity];
             private final ICopyEvaluator[] evals = new ICopyEvaluator[outputArity];
             private final ArrayBackedValueStorage evalBuf = new ArrayBackedValueStorage();
             private final RecordDescriptor inOutRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(),
                     0);
-            private final FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inOutRecDesc);
+            private final FrameTupleAccessor accessor = new FrameTupleAccessor(inOutRecDesc);
             private final FrameTupleReference frameTuple = new FrameTupleReference();
 
-            private final FrameTupleAppender tupleAppender = new FrameTupleAppender(ctx.getFrameSize());
+            private final FrameTupleAppender tupleAppender = new FrameTupleAppender();
             private final ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(inOutRecDesc.getFieldCount());
             private final DataOutput tupleDos = tupleBuilder.getDataOutput();
 
@@ -80,9 +82,8 @@ public class PartitioningSplitOperatorDescriptor extends AbstractSingleActivityO
                 // Flush (possibly not full) buffers that have data, and close writers.
                 for (int i = 0; i < outputArity; i++) {
                     tupleAppender.reset(writeBuffers[i], false);
-                    if (tupleAppender.getTupleCount() > 0) {
-                        FrameUtils.flushFrame(writeBuffers[i], writers[i]);
-                    }
+                    // ? by JF why didn't clear the buffer ?
+                    tupleAppender.flush(writers[i], false);
                     writers[i].close();
                 }
             }
@@ -133,17 +134,8 @@ public class PartitioningSplitOperatorDescriptor extends AbstractSingleActivityO
                 } catch (IOException e) {
                     throw new HyracksDataException(e);
                 }
-                // Append to frame.
-                tupleAppender.reset(writeBuffers[outputIndex], false);
-                if (!tupleAppender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
-                        tupleBuilder.getSize())) {
-                    FrameUtils.flushFrame(writeBuffers[outputIndex], writers[outputIndex]);
-                    tupleAppender.reset(writeBuffers[outputIndex], true);
-                    if (!tupleAppender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
-                            tupleBuilder.getSize())) {
-                        throw new IllegalStateException();
-                    }
-                }
+                FrameUtils.appendToWriter(writers[outputIndex], tupleAppender, tupleBuilder.getFieldEndOffsets(),
+                        tupleBuilder.getByteArray(), 0, tupleBuilder.getSize());
             }
 
             @Override
@@ -153,7 +145,7 @@ public class PartitioningSplitOperatorDescriptor extends AbstractSingleActivityO
                 }
                 // Create write buffers.
                 for (int i = 0; i < outputArity; i++) {
-                    writeBuffers[i] = ctx.allocateFrame();
+                    writeBuffers[i] = new VSizeFrame(ctx);
                     // Make sure to clear all buffers, since we are reusing the tupleAppender.
                     tupleAppender.reset(writeBuffers[i], true);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/SinkWriterRuntime.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/SinkWriterRuntime.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/SinkWriterRuntime.java
index 37ad4a9..d26d090 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/SinkWriterRuntime.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/SinkWriterRuntime.java
@@ -41,7 +41,7 @@ public class SinkWriterRuntime extends AbstractOneInputSinkPushRuntime {
         this.ctx = ctx;
         this.printStream = printStream;
         this.inputRecordDesc = inputRecordDesc;
-        this.tAccess = new FrameTupleAccessor(ctx.getFrameSize(), inputRecordDesc);
+        this.tAccess = new FrameTupleAccessor(inputRecordDesc);
     }
 
     public SinkWriterRuntime(IAWriter writer, IHyracksTaskContext ctx, PrintStream printStream,
@@ -54,7 +54,7 @@ public class SinkWriterRuntime extends AbstractOneInputSinkPushRuntime {
     public void open() throws HyracksDataException {
         if (first) {
             first = false;
-            tAccess = new FrameTupleAccessor(ctx.getFrameSize(), inputRecordDesc);
+            tAccess = new FrameTupleAccessor(inputRecordDesc);
             try {
                 writer.init();
             } catch (AlgebricksException e) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
----------------------------------------------------------------------
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
index 17603e7..4ed33f4 100644
--- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/StreamSelectRuntimeFactory.java
@@ -22,6 +22,7 @@ import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFieldFramePushRuntime;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -31,26 +32,24 @@ import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.data.std.api.IPointable;
 import edu.uci.ics.hyracks.data.std.primitive.VoidPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 public class StreamSelectRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
 
-    private IScalarEvaluatorFactory cond;
+    private final IScalarEvaluatorFactory cond;
 
-    private IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory;
+    private final IBinaryBooleanInspectorFactory binaryBooleanInspectorFactory;
 
-    private boolean retainNull;
+    private final boolean retainNull;
 
-    private int nullPlaceholderVariableIndex;
+    private final int nullPlaceholderVariableIndex;
 
-    private INullWriterFactory nullWriterFactory;
+    private final INullWriterFactory nullWriterFactory;
 
     /**
      * @param cond
-     * @param projectionList
-     *            if projectionList is null, then no projection is performed
+     * @param projectionList               if projectionList is null, then no projection is performed
      * @param retainNull
      * @param nullPlaceholderVariableIndex
      * @param nullWriterFactory
@@ -75,7 +74,7 @@ public class StreamSelectRuntimeFactory extends AbstractOneInputOneOutputRuntime
     @Override
     public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(final IHyracksTaskContext ctx) {
         final IBinaryBooleanInspector bbi = binaryBooleanInspectorFactory.createBinaryBooleanInspector(ctx);
-        return new AbstractOneInputOneOutputOneFramePushRuntime() {
+        return new AbstractOneInputOneOutputOneFieldFramePushRuntime() {
             private IPointable p = VoidPointable.FACTORY.createPointable();
             private IScalarEvaluator eval;
             private INullWriter nullWriter = null;
@@ -122,35 +121,11 @@ public class StreamSelectRuntimeFactory extends AbstractOneInputOneOutputRuntime
                         }
                     } else {
                         if (retainNull) {
-                            //keep all field values as is except setting nullPlaceholderVariable field to null
-                            int i = 0;
-                            int tryCount = 0;
-                            while (true) {
-                                for (i = 0; i < tRef.getFieldCount(); i++) {
-                                    if (i == nullPlaceholderVariableIndex) {
-                                        if (!appender.appendField(nullTupleBuilder.getByteArray(), 0,
-                                                nullTupleBuilder.getSize())) {
-                                            FrameUtils.flushFrame(frame, writer);
-                                            appender.reset(frame, true);
-                                            break;
-                                        }
-                                    } else {
-                                        if (!appender.appendField(tAccess, t, i)) {
-                                            FrameUtils.flushFrame(frame, writer);
-                                            appender.reset(frame, true);
-                                            break;
-                                        }
-                                    }
-                                }
-
-                                if (i == tRef.getFieldCount()) {
-                                    break;
+                            for (int i = 0; i < tRef.getFieldCount(); i++) {
+                                if (i == nullPlaceholderVariableIndex) {
+                                    appendField(nullTupleBuilder.getByteArray(), 0, nullTupleBuilder.getSize());
                                 } else {
-                                    tryCount++;
-                                    if (tryCount == 2) {
-                                        throw new IllegalStateException(
-                                                "Could not write frame (AbstractOneInputOneOutputOneFramePushRuntime).");
-                                    }
+                                    appendField(tAccess, t, i);
                                 }
                             }
                         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FixedSizeFrame.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FixedSizeFrame.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FixedSizeFrame.java
new file mode 100644
index 0000000..ee92084
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FixedSizeFrame.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FixedSizeFrame implements IFrame{
+
+    private final ByteBuffer buffer;
+
+    public FixedSizeFrame(ByteBuffer buffer){
+        this.buffer = buffer;
+    }
+
+    @Override
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    @Override
+    public void ensureFrameSize(int frameSize) throws HyracksDataException {
+        throw new HyracksDataException("FixedSizeFrame doesn't support capacity changes");
+    }
+
+    @Override
+    public void resize(int frameSize) throws HyracksDataException {
+        throw new HyracksDataException("FixedSizeFrame doesn't support capacity changes");
+    }
+
+    @Override
+    public int getFrameSize() {
+        return buffer.capacity();
+    }
+
+    @Override
+    public int getMinSize() {
+        return buffer.capacity() / FrameHelper.deserializeNumOfMinFrame(buffer, 0);
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        buffer.clear();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java
index 0dc97bc..176f23e 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java
@@ -15,9 +15,34 @@
 package edu.uci.ics.hyracks.api.comm;
 
 public interface FrameConstants {
-    public static final int SIZE_LEN = 4;
+    /**
+     * We use 4bytes to store the tupleCount at the end of the Frame.
+     */
+    int SIZE_LEN = 4;
 
-    public static final boolean DEBUG_FRAME_IO = false;
+    /**
+     * The offset of the frame_count which is one byte indicate how many initial_frames contained in current frame.
+     * The actual frameSize = frame_count * intitialFrameSize(given by user)
+     */
+    int META_DATA_FRAME_COUNT_OFFSET = 0;
+
+    /**
+     * The start offset of the tuple data. The first byte is used to store the frame_count
+     */
+    int TUPLE_START_OFFSET = 1;
+
+    /**
+     * Since we use one byte to store the frame_count, the max frame_count is 255.
+     */
+    int MAX_NUM_MINFRAME = 255;
+
+    /**
+     * Indicate the total size of the meta data.
+     */
+    int META_DATA_LEN = SIZE_LEN  + TUPLE_START_OFFSET;
+
+    boolean DEBUG_FRAME_IO = false;
+
+    int FRAME_FIELD_MAGIC = 0x12345678;
 
-    public static final int FRAME_FIELD_MAGIC = 0x12345678;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java
index a6774c7..2376d2e 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java
@@ -14,8 +14,65 @@
  */
 package edu.uci.ics.hyracks.api.comm;
 
+import java.nio.ByteBuffer;
+
 public class FrameHelper {
     public static int getTupleCountOffset(int frameSize) {
-        return frameSize - 4;
+        return frameSize - FrameConstants.SIZE_LEN;
+    }
+
+    /**
+     * The actual frameSize = frameCount * intitialFrameSize
+     * This method is used to put that frameCount into the first byte of the frame buffer.
+     * @param outputFrame
+     * @param numberOfMinFrame
+     */
+    public static void serializeFrameSize(ByteBuffer outputFrame, byte numberOfMinFrame) {
+        serializeFrameSize(outputFrame, 0, numberOfMinFrame);
+    }
+
+    public static void serializeFrameSize(ByteBuffer outputFrame, int start, byte numberOfMinFrame) {
+        outputFrame.array()[start + FrameConstants.META_DATA_FRAME_COUNT_OFFSET] = (byte) (numberOfMinFrame & 0xff);
+    }
+
+    public static byte deserializeNumOfMinFrame(ByteBuffer frame) {
+        return deserializeNumOfMinFrame(frame, 0);
+    }
+
+    public static byte deserializeNumOfMinFrame(ByteBuffer buffer, int start) {
+        return (byte) (buffer.array()[start + FrameConstants.META_DATA_FRAME_COUNT_OFFSET] & 0xff);
+    }
+
+    /**
+     * Add one tuple requires
+     * 4bytes to store the tuple offset
+     * 4bytes * |fields| to store the relative offset of each field
+     * nbytes the actual data.
+     * If the tupleLength includes the field slot, please set the fieldCount = 0
+     */
+    public static int calcSpaceInFrame(int fieldCount, int tupleLength) {
+        return 4 + fieldCount * 4 + tupleLength;
+    }
+
+    /**
+     * A faster way of calculating the ceiling
+     *
+     * @param fieldCount   please set fieldCount to 0 if the tupleLength includes the fields' length
+     * @param tupleLength
+     * @param minFrameSize
+     * @return
+     */
+    public static int calcAlignedFrameSizeToStore(int fieldCount, int tupleLength, int minFrameSize) {
+        assert fieldCount >= 0 && tupleLength >= 0 && minFrameSize > 0;
+        return (1 + (calcSpaceInFrame(fieldCount, tupleLength) + FrameConstants.META_DATA_LEN - 1) / minFrameSize)
+                * minFrameSize;
+    }
+
+    public static void clearRemainingFrame(ByteBuffer buffer, int position) {
+        buffer.array()[position] = 0;
+    }
+
+    public static boolean hasBeenCleared(ByteBuffer buffer, int position) {
+        return deserializeNumOfMinFrame(buffer, position) == 0;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrame.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrame.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrame.java
new file mode 100644
index 0000000..ccbbb0d
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrame.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrame {
+
+    ByteBuffer getBuffer();
+
+    /**
+     * Make sure the frameSize is bigger or equal to the given size
+     *
+     * @param frameSize
+     * @throws HyracksDataException
+     */
+    void ensureFrameSize(int frameSize) throws HyracksDataException;
+
+    /**
+     *
+     * Expand of shrink the inner buffer to make the size exactly equal to {@code frameSize}
+     * @param frameSize
+     */
+    void resize(int frameSize) throws HyracksDataException;
+
+    /**
+     * Return the size of frame in bytes
+     *
+     * @return
+     */
+    int getFrameSize();
+
+    /**
+     * Return the minimum frame size which should read from the configuration file given by user
+     *
+     * @return
+     */
+    int getMinSize();
+
+    /**
+     * Reset the status of buffer, prepare to the next round of read/write
+     */
+    void reset() throws HyracksDataException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameAppender.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameAppender.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameAppender.java
new file mode 100644
index 0000000..89b2bba
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameAppender.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameAppender {
+    /**
+     * Reset to attach to a new frame.
+     *
+     * @param frame the new frame
+     * @param clear indicate whether we need to clear this new frame
+     * @throws HyracksDataException
+     */
+    void reset(IFrame frame, boolean clear) throws HyracksDataException;
+
+    /**
+     * Get how many tuples in current frame.
+     *
+     * @return
+     */
+    int getTupleCount();
+
+    /**
+     * Get the ByteBuffer which contains the frame data.
+     *
+     * @return
+     */
+    ByteBuffer getBuffer();
+
+    /**
+     * Flush the frame content to the given writer.
+     * Clear the inner buffer after flush if {@code clear} is <code>true</code>.
+     *
+     * @param outWriter the output writer
+     * @param clear     indicate whether to clear the inside frame after flushed or not.
+     * @throws HyracksDataException
+     */
+    void flush(IFrameWriter outWriter, boolean clear) throws HyracksDataException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameFieldAppender.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameFieldAppender.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameFieldAppender.java
new file mode 100644
index 0000000..f66248f
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameFieldAppender.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * The IFrameFieldAppender is used to append the data into frame field by field.
+ */
+public interface IFrameFieldAppender extends IFrameAppender {
+
+    /**
+     * Append the field stored in {@code bytes} into the current frame.
+     *
+     * @param bytes  the byte array that stores the field data
+     * @param offset the offset of the field data
+     * @param length the length of the field data
+     * @return true if the current frame has enough space to hold the field data, otherwise return false.
+     * @throws HyracksDataException
+     */
+    boolean appendField(byte[] bytes, int offset, int length) throws HyracksDataException;
+
+    /**
+     * Append the field of {@code fid} from the tuple {@code tid} whose information is stored in the {@code accessor}
+     * into the current frame.
+     *
+     * @param accessor tupleAccessor
+     * @param tid      tuple id in tupleAccessor
+     * @param fid      field id of the tuple {@code tid}
+     * @return true if the current frame has enough space to hold the field data, otherwise return false.
+     * @throws HyracksDataException
+     */
+    boolean appendField(IFrameTupleAccessor accessor, int tid, int fid) throws HyracksDataException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java
index c72782a..cd3c5ab 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java
@@ -14,14 +14,12 @@
  */
 package edu.uci.ics.hyracks.api.comm;
 
-import java.nio.ByteBuffer;
-
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public interface IFrameReader {
-    public void open() throws HyracksDataException;
+    void open() throws HyracksDataException;
 
-    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException;
+    boolean nextFrame(IFrame frame) throws HyracksDataException;
 
-    public void close() throws HyracksDataException;
+    void close() throws HyracksDataException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAccessor.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAccessor.java
index ee34add..130704f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAccessor.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAccessor.java
@@ -17,23 +17,28 @@ package edu.uci.ics.hyracks.api.comm;
 import java.nio.ByteBuffer;
 
 public interface IFrameTupleAccessor {
-    public int getFieldCount();
+    int getFieldCount();
 
-    public int getFieldSlotsLength();
+    int getFieldSlotsLength();
 
-    public int getFieldEndOffset(int tupleIndex, int fIdx);
+    int getFieldEndOffset(int tupleIndex, int fIdx);
 
-    public int getFieldStartOffset(int tupleIndex, int fIdx);
+    int getFieldStartOffset(int tupleIndex, int fIdx);
 
-    public int getFieldLength(int tupleIndex, int fIdx);
+    int getFieldLength(int tupleIndex, int fIdx);
 
-    public int getTupleEndOffset(int tupleIndex);
+    int getTupleLength(int tupleIndex);
 
-    public int getTupleStartOffset(int tupleIndex);
+    int getTupleEndOffset(int tupleIndex);
 
-    public int getTupleCount();
+    int getTupleStartOffset(int tupleIndex);
 
-    public ByteBuffer getBuffer();
+    int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx);
+
+    int getTupleCount();
+
+    ByteBuffer getBuffer();
+
+    void reset(ByteBuffer buffer);
 
-    public void reset(ByteBuffer buffer);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAppender.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAppender.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAppender.java
new file mode 100644
index 0000000..4da2afc
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAppender.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameTupleAppender extends IFrameAppender {
+
+    boolean append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException;
+
+    boolean append(int[] fieldSlots, byte[] bytes, int offset, int length) throws HyracksDataException;
+
+    boolean append(byte[] bytes, int offset, int length) throws HyracksDataException;
+
+    boolean appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length) throws HyracksDataException;
+
+    boolean append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset) throws HyracksDataException;
+
+    boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1,
+            int tIndex1) throws HyracksDataException;
+
+    boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, int[] fieldSlots1, byte[] bytes1,
+            int offset1, int dataLen1) throws HyracksDataException;
+
+    boolean appendProjection(IFrameTupleAccessor accessor, int tIndex, int[] fields) throws HyracksDataException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java
index 8e35dda..538759e 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java
@@ -45,7 +45,7 @@ import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
  * Note: If the call to {@link IFrameWriter#open()} failed, the {@link IFrameWriter#close()} is not called by the producer. So an exceptional
  * return from the {@link IFrameWriter#open()} call must clean up all partially
  * allocated resources.
- * 
+ *
  * @author vinayakb
  */
 public interface IFrameWriter {
@@ -56,9 +56,8 @@ public interface IFrameWriter {
 
     /**
      * Provide data to the stream of this {@link IFrameWriter}.
-     * 
-     * @param buffer
-     *            - Buffer containing data.
+     *
+     * @param buffer - Buffer containing data.
      * @throws HyracksDataException
      */
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException;
@@ -66,14 +65,14 @@ public interface IFrameWriter {
     /**
      * Indicate that a failure was encountered and the current stream is to be
      * aborted.
-     * 
+     *
      * @throws HyracksDataException
      */
     public void fail() throws HyracksDataException;
 
     /**
      * Close this {@link IFrameWriter} and give up all resources.
-     * 
+     *
      * @throws HyracksDataException
      */
     public void close() throws HyracksDataException;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NoShrinkVSizeFrame.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NoShrinkVSizeFrame.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NoShrinkVSizeFrame.java
new file mode 100644
index 0000000..902ae75
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NoShrinkVSizeFrame.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import edu.uci.ics.hyracks.api.context.IHyracksFrameMgrContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NoShrinkVSizeFrame extends VSizeFrame {
+    public NoShrinkVSizeFrame(IHyracksFrameMgrContext ctx) throws HyracksDataException {
+        super(ctx);
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        buffer.clear();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/VSizeFrame.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/VSizeFrame.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/VSizeFrame.java
new file mode 100644
index 0000000..a5a7f19
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/VSizeFrame.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksFrameMgrContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Variable size frame. The buffer inside can be resized.
+ */
+public class VSizeFrame implements IFrame {
+
+    protected final int minFrameSize;
+    protected IHyracksFrameMgrContext ctx;
+    protected ByteBuffer buffer;
+
+    public VSizeFrame(IHyracksFrameMgrContext ctx) throws HyracksDataException {
+        this(ctx, ctx.getInitialFrameSize());
+    }
+
+    public VSizeFrame(IHyracksFrameMgrContext ctx, int frameSize) throws HyracksDataException {
+        this.minFrameSize = ctx.getInitialFrameSize();
+        this.ctx = ctx;
+        buffer = ctx.allocateFrame(frameSize);
+    }
+
+    public ByteBuffer getBuffer() {
+        return buffer;
+    }
+
+    @Override
+    public void ensureFrameSize(int newSize) throws HyracksDataException {
+        if (newSize > getFrameSize()) {
+            buffer = ctx.reallocateFrame(buffer, newSize, true);
+        }
+    }
+
+    @Override
+    public void resize(int frameSize) throws HyracksDataException {
+        if (getFrameSize() != frameSize) {
+            buffer = ctx.reallocateFrame(buffer, frameSize, false);
+        }
+    }
+
+    @Override
+    public int getFrameSize() {
+        return buffer.capacity();
+    }
+
+    @Override
+    public int getMinSize() {
+        return minFrameSize;
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        resize(minFrameSize);
+        buffer.clear();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
index eddc4df..d60ff6e 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
@@ -14,17 +14,9 @@
  */
 package edu.uci.ics.hyracks.api.context;
 
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.IIOManager;
 
-public interface IHyracksCommonContext {
-    public int getFrameSize();
+public interface IHyracksCommonContext extends IHyracksFrameMgrContext{
 
     public IIOManager getIOManager();
-
-    public ByteBuffer allocateFrame() throws HyracksDataException;
-    
-    public void deallocateFrames(int frameCount);
 }


[06/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGenerator.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGenerator.java
deleted file mode 100644
index d78af12..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGenerator.java
+++ /dev/null
@@ -1,283 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
-
-/**
- * @author pouria This class implements the run generator for sorting with
- *         replacement selection, where there is no limit on the output, i.e.
- *         the whole data should be sorted. A SortMinHeap is used as the
- *         selectionTree to decide the order of writing tuples into the runs,
- *         while memory manager is based on a binary search tree to allocate
- *         tuples in the memory. The overall process is as follows: - Read the
- *         input data frame by frame. For each tuple T in the current frame: -
- *         Try to allocate a memory slot for writing T along with the attached
- *         header/footer (for memory management purpose) - If T can not be
- *         allocated, try to output as many tuples, currently resident in
- *         memory, as needed so that a free slot, large enough to hold T, gets
- *         created. MinHeap decides about which tuple should be sent to the
- *         output at each step. - Write T into the memory - Calculate the runID
- *         of T (based on the last output tuple for the current run). It is
- *         either the current run or the next run. Also calculate Poorman's
- *         Normalized Key (PNK) for T, to make comparisons faster later. -
- *         Create a heap element for T, containing: its runID, the slot pointer
- *         to its memory location, and its PNK. - Insert the created heap
- *         element into the heap - Upon closing, write all the tuples, currently
- *         resident in memory, into their corresponding run(s). Again min heap
- *         decides about which tuple is the next for output.
- *         OptimizedSortOperatorDescriptor will merge the generated runs, to
- *         generate the final sorted output of the data.
- */
-public class OptimizedExternalSortRunGenerator implements IRunGenerator {
-    private final IHyracksTaskContext ctx;
-    private final int[] sortFields;
-    private final INormalizedKeyComputer nkc;
-    private final IBinaryComparatorFactory[] comparatorFactories;
-    private final IBinaryComparator[] comparators;
-    private final RecordDescriptor recordDescriptor;
-    private final List<IFrameReader> runs;
-
-    private ISelectionTree sTree;
-    private IMemoryManager memMgr;
-
-    private final int memSize;
-    private FrameTupleAccessor inputAccessor; // Used to read tuples in
-                                              // nextFrame()
-    private FrameTupleAppender outputAppender; // Used to write tuple to the
-                                               // dedicated output buffer
-    private ByteBuffer outputBuffer; // Dedicated output buffer to write tuples
-                                     // into run(s)
-    private FrameTupleAccessor lastRecordAccessor; // Used to read last output
-                                                   // record from the output
-                                                   // buffer
-    private int lastTupleIx; // Holds index of last output tuple in the
-                             // dedicated output buffer
-    private Slot allocationPtr; // Contains the ptr to the allocated memory slot
-                                // by the memory manager for the new tuple
-    private Slot outputedTuple; // Contains the ptr to the next tuple chosen by
-                                // the selectionTree to output
-    private int[] sTreeTop;
-
-    private RunFileWriter writer;
-
-    private boolean newRun;
-    private int curRunId;
-
-    public OptimizedExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
-            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDesc, int memSize) {
-        this.ctx = ctx;
-        this.sortFields = sortFields;
-        nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
-        this.comparatorFactories = comparatorFactories;
-        comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
-        this.recordDescriptor = recordDesc;
-        this.runs = new LinkedList<IFrameReader>();
-        this.memSize = memSize;
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        runs.clear();
-        inputAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        outputAppender = new FrameTupleAppender(ctx.getFrameSize());
-        outputBuffer = ctx.allocateFrame();
-        outputAppender.reset(outputBuffer, true);
-        lastRecordAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-
-        this.memMgr = new BSTMemMgr(ctx, memSize);
-        this.sTree = new SortMinHeap(ctx, sortFields, comparatorFactories, recordDescriptor, memMgr);
-        this.allocationPtr = new Slot();
-        this.outputedTuple = new Slot();
-        this.sTreeTop = new int[] { -1, -1, -1, -1 };
-        curRunId = -1;
-        openNewRun();
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        inputAccessor.reset(buffer);
-        byte[] bufferArray = buffer.array();
-        int tupleCount = inputAccessor.getTupleCount();
-        for (int i = 0; i < tupleCount; ++i) {
-            allocationPtr.clear();
-            int tLength = inputAccessor.getTupleEndOffset(i) - inputAccessor.getTupleStartOffset(i);
-            memMgr.allocate(tLength, allocationPtr);
-            while (allocationPtr.isNull()) {
-                int unAllocSize = -1;
-                while (unAllocSize < tLength) {
-                    unAllocSize = outputRecord();
-                    if (unAllocSize < 1) {
-                        throw new HyracksDataException(
-                                "Unable to allocate space for the new tuple, while there is no more tuple to output");
-                    }
-                }
-                memMgr.allocate(tLength, allocationPtr);
-            }
-            memMgr.writeTuple(allocationPtr.getFrameIx(), allocationPtr.getOffset(), inputAccessor, i);
-            int runId = getRunId(inputAccessor, i);
-            int pnk = getPNK(inputAccessor, i, bufferArray);
-            int[] entry = new int[] { runId, allocationPtr.getFrameIx(), allocationPtr.getOffset(), pnk };
-            sTree.insert(entry);
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        while (!sTree.isEmpty()) { // Outputting remaining elements in the
-                                   // selectionTree
-            outputRecord();
-        }
-        if (outputAppender.getTupleCount() > 0) { // Writing out very last
-                                                  // resident records to file
-            FrameUtils.flushFrame(outputBuffer, writer);
-        }
-        outputAppender.reset(outputBuffer, true);
-        writer.close();
-        runs.add(writer.createReader());
-        memMgr.close();
-    }
-
-    public List<IFrameReader> getRuns() {
-        return runs;
-    }
-
-    private int outputRecord() throws HyracksDataException {
-        outputedTuple.clear();
-        sTree.getMin(sTreeTop);
-        if (!isEntryValid(sTreeTop)) {
-            throw new HyracksDataException("Invalid outputed tuple (Top of the selection tree is invalid)");
-        }
-
-        if (sTreeTop[SortMinHeap.RUN_ID_IX] != curRunId) { // We need to switch
-                                                           // runs
-            openNewRun();
-        }
-
-        int tFrameIx = sTreeTop[SortMinHeap.FRAME_IX];
-        int tOffset = sTreeTop[SortMinHeap.OFFSET_IX];
-        if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) { // Can not
-                                                                    // append to
-                                                                    // the
-                                                                    // tupleAppender
-            FrameUtils.flushFrame(outputBuffer, writer);
-            outputAppender.reset(outputBuffer, true);
-            if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) {
-                throw new HyracksDataException("Can not append to the ouput buffer in sort");
-            }
-            lastTupleIx = 0;
-        } else {
-            lastTupleIx++;
-        }
-        outputedTuple.set(tFrameIx, tOffset);
-        newRun = false;
-        return memMgr.unallocate(outputedTuple);
-
-    }
-
-    private int getPNK(FrameTupleAccessor fta, int tIx, byte[] buffInArray) {
-        // Moved buffInArray out for better performance (not converting for each and every tuple)
-        int sfIdx = sortFields[0];
-        int tStart = fta.getTupleStartOffset(tIx);
-        int f0StartRel = fta.getFieldStartOffset(tIx, sfIdx);
-        int f0EndRel = fta.getFieldEndOffset(tIx, sfIdx);
-        int f0Start = f0StartRel + tStart + fta.getFieldSlotsLength();
-        return (nkc == null ? 0 : nkc.normalize(buffInArray, f0Start, f0EndRel - f0StartRel));
-    }
-
-    private int getRunId(FrameTupleAccessor fta, int tupIx) throws HyracksDataException {
-        // Comparing current record to last output record, it decides about current record's runId
-        if (newRun) { // Very first record for a new run
-            return curRunId;
-        }
-
-        byte[] lastRecBuff = outputBuffer.array();
-        lastRecordAccessor.reset(outputBuffer);
-        int lastStartOffset = lastRecordAccessor.getTupleStartOffset(lastTupleIx);
-
-        ByteBuffer fr2 = fta.getBuffer();
-        byte[] curRecBuff = fr2.array();
-        int r2StartOffset = fta.getTupleStartOffset(tupIx);
-
-        for (int f = 0; f < comparators.length; ++f) {
-            int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : outputBuffer.getInt(lastStartOffset + (fIdx - 1) * 4);
-            int f1End = outputBuffer.getInt(lastStartOffset + fIdx * 4);
-            int s1 = lastStartOffset + lastRecordAccessor.getFieldSlotsLength() + f1Start;
-            int l1 = f1End - f1Start;
-            int f2Start = fIdx == 0 ? 0 : fr2.getInt(r2StartOffset + (fIdx - 1) * 4);
-            int f2End = fr2.getInt(r2StartOffset + fIdx * 4);
-            int s2 = r2StartOffset + fta.getFieldSlotsLength() + f2Start;
-            int l2 = f2End - f2Start;
-            int c = comparators[f].compare(lastRecBuff, s1, l1, curRecBuff, s2, l2);
-            if (c != 0) {
-                if (c <= 0) {
-                    return curRunId;
-                } else {
-                    return (curRunId + 1);
-                }
-            }
-        }
-        return curRunId;
-    }
-
-    private void openNewRun() throws HyracksDataException {
-        if (writer != null) { // There is a prev run, so flush its tuples and
-                              // close it first
-            if (outputAppender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(outputBuffer, writer);
-            }
-            outputAppender.reset(outputBuffer, true);
-            writer.close();
-            runs.add(writer.createReader());
-        }
-
-        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
-                ExternalSortRunGenerator.class.getSimpleName());
-        writer = new RunFileWriter(file, ctx.getIOManager());
-        writer.open();
-        curRunId++;
-        newRun = true;
-        lastTupleIx = -1;
-    }
-
-    private boolean isEntryValid(int[] entry) {
-        return ((entry[SortMinHeap.RUN_ID_IX] > -1) && (entry[SortMinHeap.FRAME_IX] > -1) && (entry[SortMinHeap.OFFSET_IX] > -1));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGeneratorWithLimit.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGeneratorWithLimit.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGeneratorWithLimit.java
deleted file mode 100644
index 5b01fb8..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGeneratorWithLimit.java
+++ /dev/null
@@ -1,436 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
-
-/**
- * @author pouria
- *         This class implements the run generator for sorting with replacement
- *         selection, where there is a limit on the output, i.e. we are looking
- *         for top-k tuples (first k smallest tuples w.r.t sorting keys).
- *         A SortMinMaxHeap is used as the selectionTree to decide the order of
- *         writing tuples into the runs, and also to prune tuples (if possible).
- *         Memory manager is based on a binary search tree and is used to
- *         allocate memory slots for tuples.
- *         The overall process is as follows (Assuming that the limit is K):
- *         - Read the input data frame by frame. For each tuple T in the current
- *         frame:
- *         - If currentRun R has reached the limit of K on the size, and (T >
- *         maximum tuple of R), then ignore T.
- *         - Otherwise, try to allocate a memory slot for writing T along with
- *         the attached header/footer (for memory management purpose)
- *         - If T can not be allocated, try to output as many tuples, currently
- *         resident in memory, as needed so that a free slot, large enough to
- *         hold T, gets created. MinMaxHeap decides about which tuple should be
- *         sent to the output at each step.
- *         - Write T into memory.
- *         - Calculate the runID of T (based on the last output tuple for the
- *         current run). It is either the current run or the next run. Also
- *         calculate Poorman's Normalized Key (PNK) for T, to make comparisons
- *         faster later.
- *         - Create an heap element for T, containing its runID, the slot ptr to
- *         its memory location, and its PNK.
- *         - If runID is the nextRun, insert the heap element into the heap, and
- *         increment the size of nextRun.
- *         - If runID is the currentRun, then:
- *         - If currentRun has not hit the limit of k, insert the element into
- *         the heap, and increase currentRun size. - Otherwise, currentRun has
- *         hit the limit of K, while T is less than the max. So discard the
- *         current max for the current run (by poping it from the heap and
- *         unallocating its memory location) and insert the heap element into
- *         the heap. No need to change the currentRun size as we are replacing
- *         an old element (the old max) with T.
- *         - Upon closing, write all the tuples, currently resident in memory,
- *         into their corresponding run(s).
- *         - Note that upon opening a new Run R, if size of R (based on stats)
- *         is S and (S > K), then (S-K) current maximum tuples of R (which are
- *         resident in memory) get discarded at the beginning. MinMax heap can
- *         be used to find these tuples.
- */
-public class OptimizedExternalSortRunGeneratorWithLimit implements IRunGenerator {
-
-    private final IHyracksTaskContext ctx;
-    private final int[] sortFields;
-    private final INormalizedKeyComputer nkc;
-    private final IBinaryComparatorFactory[] comparatorFactories;
-    private final IBinaryComparator[] comparators;
-    private final RecordDescriptor recordDescriptor;
-    private final List<IFrameReader> runs;
-
-    private ISelectionTree sTree;
-    private IMemoryManager memMgr;
-
-    private final int memSize;
-    private FrameTupleAccessor inputAccessor; // Used to read tuples in
-                                              // nextFrame()
-    private FrameTupleAppender outputAppender; // Used to write tuple to the
-                                               // dedicated output buffer
-    private ByteBuffer outputBuffer; // Dedicated output buffer to write tuples
-                                     // into run(s)
-    private FrameTupleAccessor lastRecordAccessor; // Used to read last output
-                                                   // record from the output
-                                                   // buffer
-    private FrameTupleAccessor fta2; // Used to read max record
-    private final int outputLimit;
-    private int curRunSize;
-    private int nextRunSize;
-    private int lastTupleIx; // Holds index of last output tuple in the
-                             // dedicated output buffer
-    private Slot allocationPtr; // Contains the ptr to the allocated memory slot
-                                // by the memory manager for the new tuple
-    private Slot outputedTuple; // Contains the ptr to the next tuple chosen by
-                                // the selectionTree to output
-    private Slot discard;
-    private int[] sTreeTop;
-    private int[] peek;
-    private RunFileWriter writer;
-    private boolean newRun;
-    private int curRunId;
-
-    public OptimizedExternalSortRunGeneratorWithLimit(IHyracksTaskContext ctx, int[] sortFields,
-            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDesc, int memSize, int limit) {
-
-        this.ctx = ctx;
-        this.sortFields = sortFields;
-        nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
-        this.comparatorFactories = comparatorFactories;
-        comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
-        this.recordDescriptor = recordDesc;
-        this.runs = new LinkedList<IFrameReader>();
-        this.memSize = memSize;
-
-        this.outputLimit = limit;
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        runs.clear();
-        inputAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        outputAppender = new FrameTupleAppender(ctx.getFrameSize());
-        outputBuffer = ctx.allocateFrame();
-        outputAppender.reset(outputBuffer, true);
-        lastRecordAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        this.memMgr = new BSTMemMgr(ctx, memSize);
-        this.sTree = new SortMinMaxHeap(ctx, sortFields, comparatorFactories, recordDescriptor, memMgr);
-        this.allocationPtr = new Slot();
-        this.outputedTuple = new Slot();
-        this.sTreeTop = new int[] { -1, -1, -1, -1 };
-        this.peek = new int[] { -1, -1, -1, -1 };
-        this.discard = new Slot();
-
-        curRunId = -1;
-        curRunSize = 0;
-        nextRunSize = 0;
-        openNewRun();
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        inputAccessor.reset(buffer);
-        byte[] bufferArray = buffer.array();
-        int tupleCount = inputAccessor.getTupleCount();
-        for (int i = 0; i < tupleCount; i++) {
-            if (curRunSize >= outputLimit) {
-                sTree.peekMax(peek);
-                if (isEntryValid(peek)
-                        && compareRecords(inputAccessor, i, peek[SortMinMaxHeap.FRAME_IX],
-                                peek[SortMinMaxHeap.OFFSET_IX]) >= 0) {
-                    continue;
-                }
-            }
-
-            allocationPtr.clear();
-            int tLength = inputAccessor.getTupleEndOffset(i) - inputAccessor.getTupleStartOffset(i);
-            memMgr.allocate(tLength, allocationPtr);
-            while (allocationPtr.isNull()) {
-                int unAllocSize = -1;
-                while (unAllocSize < tLength) {
-                    unAllocSize = outputRecord();
-                    if (unAllocSize < 1) {
-                        throw new HyracksDataException(
-                                "Unable to allocate space for the new tuple, while there is no more tuple to output");
-                    }
-                }
-                memMgr.allocate(tLength, allocationPtr);
-            }
-
-            int pnk = getPNK(inputAccessor, i, bufferArray);
-            int runId = getRunId(inputAccessor, i);
-            if (runId != curRunId) { // tuple belongs to the next run
-                memMgr.writeTuple(allocationPtr.getFrameIx(), allocationPtr.getOffset(), inputAccessor, i);
-                int[] entry = new int[] { runId, allocationPtr.getFrameIx(), allocationPtr.getOffset(), pnk };
-                sTree.insert(entry);
-                nextRunSize++;
-                continue;
-            }
-            // belongs to the current run
-            if (curRunSize < outputLimit) {
-                memMgr.writeTuple(allocationPtr.getFrameIx(), allocationPtr.getOffset(), inputAccessor, i);
-                int[] entry = new int[] { runId, allocationPtr.getFrameIx(), allocationPtr.getOffset(), pnk };
-                sTree.insert(entry);
-                curRunSize++;
-                continue;
-            }
-
-            sTree.peekMax(peek);
-            if (compareRecords(inputAccessor, i, peek[SortMinMaxHeap.FRAME_IX], peek[SortMinMaxHeap.OFFSET_IX]) > 0) {
-                continue;
-            }
-            // replacing the max
-            sTree.getMax(peek);
-            discard.set(peek[SortMinMaxHeap.FRAME_IX], peek[SortMinMaxHeap.OFFSET_IX]);
-            memMgr.unallocate(discard);
-            memMgr.writeTuple(allocationPtr.getFrameIx(), allocationPtr.getOffset(), inputAccessor, i);
-            int[] entry = new int[] { runId, allocationPtr.getFrameIx(), allocationPtr.getOffset(), pnk };
-            sTree.insert(entry);
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        while (!sTree.isEmpty()) { // Outputting remaining elements in the
-                                   // selectionTree
-            outputRecordForClose();
-        }
-
-        if (outputAppender.getTupleCount() > 0) { // Writing out very last
-                                                  // resident records to file
-            FrameUtils.flushFrame(outputBuffer, writer);
-        }
-
-        writer.close();
-        runs.add(writer.createReader());
-        memMgr.close();
-    }
-
-    public List<IFrameReader> getRuns() {
-        return runs;
-    }
-
-    private int outputRecord() throws HyracksDataException {
-        outputedTuple.clear();
-        sTree.getMin(sTreeTop);
-        if (!isEntryValid(sTreeTop)) {
-            throw new HyracksDataException("Invalid outputed tuple (Top of the selection tree is invalid)");
-        }
-        int tFrameIx = sTreeTop[SortMinHeap.FRAME_IX];
-        int tOffset = sTreeTop[SortMinHeap.OFFSET_IX];
-        if (sTreeTop[SortMinMaxHeap.RUN_ID_IX] == curRunId) {
-            if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) { // Can
-                                                                        // not
-                                                                        // append
-                                                                        // to
-                                                                        // the
-                                                                        // tupleAppender
-                FrameUtils.flushFrame(outputBuffer, writer);
-                outputAppender.reset(outputBuffer, true);
-                if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) {
-                    throw new HyracksDataException("Can not append to the ouput buffer in sort");
-                }
-                lastTupleIx = 0;
-            } else {
-                lastTupleIx++;
-            }
-            outputedTuple.set(tFrameIx, tOffset);
-            newRun = false;
-            return memMgr.unallocate(outputedTuple);
-        }
-        // Minimum belongs to the next Run
-        openNewRun();
-        int popCount = curRunSize - outputLimit;
-        int l = 0;
-        int maxFreedSpace = 0;
-        for (int p = 0; p < popCount; p++) {
-            sTree.getMax(peek);
-            if (!isEntryValid(peek)) {
-                throw new HyracksDataException("Invalid Maximum extracted from MinMaxHeap");
-            }
-            discard.set(peek[SortMinMaxHeap.FRAME_IX], peek[SortMinMaxHeap.OFFSET_IX]);
-            l = memMgr.unallocate(discard);
-            if (l > maxFreedSpace) {
-                maxFreedSpace = l;
-            }
-            curRunSize--;
-        }
-
-        if (maxFreedSpace != 0) {
-            return maxFreedSpace;
-        }
-        // No max discarded (We just flushed out the prev run, so the output
-        // buffer should be clear)
-        if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) { // Can not
-                                                                    // append to
-                                                                    // the
-                                                                    // tupleAppender
-            throw new HyracksDataException("Can not append to the ouput buffer in sort");
-        }
-        lastTupleIx = 0;
-        outputedTuple.set(tFrameIx, tOffset);
-        newRun = false;
-        return memMgr.unallocate(outputedTuple);
-    }
-
-    private void outputRecordForClose() throws HyracksDataException {
-        sTree.getMin(sTreeTop);
-        if (!isEntryValid(sTreeTop)) {
-            throw new HyracksDataException("Invalid outputed tuple (Top of the selection tree is invalid)");
-        }
-        int tFrameIx = sTreeTop[SortMinHeap.FRAME_IX];
-        int tOffset = sTreeTop[SortMinHeap.OFFSET_IX];
-        if (sTreeTop[SortMinMaxHeap.RUN_ID_IX] != curRunId) {
-            openNewRun();
-        }
-
-        if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) { // Can not
-                                                                    // append to
-                                                                    // the
-                                                                    // tupleAppender
-            FrameUtils.flushFrame(outputBuffer, writer);
-            outputAppender.reset(outputBuffer, true);
-            if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) {
-                throw new HyracksDataException("Can not append to the ouput buffer in sort");
-            }
-        }
-    }
-
-    private int getPNK(FrameTupleAccessor fta, int tIx, byte[] buffInArray) {
-        // Moved buffInArray out for better performance (not converting for each and every tuple)
-        int sfIdx = sortFields[0];
-        int tStart = fta.getTupleStartOffset(tIx);
-        int f0StartRel = fta.getFieldStartOffset(tIx, sfIdx);
-        int f0EndRel = fta.getFieldEndOffset(tIx, sfIdx);
-        int f0Start = f0StartRel + tStart + fta.getFieldSlotsLength();
-        return (nkc == null ? 0 : nkc.normalize(buffInArray, f0Start, f0EndRel - f0StartRel));
-    }
-
-    private int getRunId(FrameTupleAccessor fta, int tupIx) throws HyracksDataException {
-        // Comparing current record to last output record, it decides about current record's runId
-        if (newRun) { // Very first record for a new run
-            return curRunId;
-        }
-
-        byte[] lastRecBuff = outputBuffer.array();
-        lastRecordAccessor.reset(outputBuffer);
-        int lastStartOffset = lastRecordAccessor.getTupleStartOffset(lastTupleIx);
-
-        ByteBuffer fr2 = fta.getBuffer();
-        byte[] curRecBuff = fr2.array();
-        int r2StartOffset = fta.getTupleStartOffset(tupIx);
-
-        for (int f = 0; f < comparators.length; ++f) {
-            int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : outputBuffer.getInt(lastStartOffset + (fIdx - 1) * 4);
-            int f1End = outputBuffer.getInt(lastStartOffset + fIdx * 4);
-            int s1 = lastStartOffset + lastRecordAccessor.getFieldSlotsLength() + f1Start;
-            int l1 = f1End - f1Start;
-            int f2Start = fIdx == 0 ? 0 : fr2.getInt(r2StartOffset + (fIdx - 1) * 4);
-            int f2End = fr2.getInt(r2StartOffset + fIdx * 4);
-            int s2 = r2StartOffset + fta.getFieldSlotsLength() + f2Start;
-            int l2 = f2End - f2Start;
-            int c = comparators[f].compare(lastRecBuff, s1, l1, curRecBuff, s2, l2);
-            if (c != 0) {
-                if (c <= 0) {
-                    return curRunId;
-                } else {
-                    return (curRunId + 1);
-                }
-            }
-        }
-        return curRunId;
-    }
-
-    // first<sec : -1
-    private int compareRecords(FrameTupleAccessor fta1, int ix1, int fix2, int offset2) throws HyracksDataException {
-        ByteBuffer buff1 = fta1.getBuffer();
-        byte[] recBuff1 = buff1.array();
-        int offset1 = fta1.getTupleStartOffset(ix1);
-
-        offset2 += BSTNodeUtil.HEADER_SIZE;
-        ByteBuffer buff2 = memMgr.getFrame(fix2);
-        fta2.reset(buff2);
-        byte[] recBuff2 = buff2.array();
-
-        for (int f = 0; f < comparators.length; ++f) {
-            int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : buff1.getInt(offset1 + (fIdx - 1) * 4);
-            int f1End = buff1.getInt(offset1 + fIdx * 4);
-            int s1 = offset1 + fta1.getFieldSlotsLength() + f1Start;
-            int l1 = f1End - f1Start;
-            int f2Start = fIdx == 0 ? 0 : buff2.getInt(offset2 + (fIdx - 1) * 4);
-            int f2End = buff2.getInt(offset2 + fIdx * 4);
-            int s2 = offset2 + fta2.getFieldSlotsLength() + f2Start;
-            int l2 = f2End - f2Start;
-            int c = comparators[f].compare(recBuff1, s1, l1, recBuff2, s2, l2);
-
-            if (c != 0) {
-                return c;
-            }
-        }
-        return 0;
-
-    }
-
-    private void openNewRun() throws HyracksDataException {
-        if (writer != null) { // There is a prev run, so flush its tuples and
-                              // close it first
-            if (outputAppender.getTupleCount() > 0) {
-                FrameUtils.flushFrame(outputBuffer, writer);
-            }
-            outputAppender.reset(outputBuffer, true);
-            writer.close();
-            runs.add(writer.createReader());
-        }
-
-        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
-                ExternalSortRunGenerator.class.getSimpleName());
-        writer = new RunFileWriter(file, ctx.getIOManager());
-        writer.open();
-        curRunId++;
-        newRun = true;
-        curRunSize = nextRunSize;
-        nextRunSize = 0;
-        lastTupleIx = -1;
-    }
-
-    private boolean isEntryValid(int[] entry) {
-        return ((entry[SortMinHeap.RUN_ID_IX] > -1) && (entry[SortMinHeap.FRAME_IX] > -1) && (entry[SortMinHeap.OFFSET_IX] > -1));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunAndMaxFrameSizePair.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunAndMaxFrameSizePair.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunAndMaxFrameSizePair.java
new file mode 100644
index 0000000..c68f1e7
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunAndMaxFrameSizePair.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+
+public class RunAndMaxFrameSizePair {
+    public IFrameReader run;
+    public int maxFrameSize;
+
+    public RunAndMaxFrameSizePair(IFrameReader run, int maxFrameSize) {
+        this.run = run;
+        this.maxFrameSize = maxFrameSize;
+    }
+
+    void updateSize(int newMaxSize){
+        this.maxFrameSize = newMaxSize;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
index cf0d0ad..f013594 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
@@ -1,23 +1,23 @@
 /*
  * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
+ *  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.
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
  */
 package edu.uci.ics.hyracks.dataflow.std.sort;
 
-import java.nio.ByteBuffer;
 import java.util.Comparator;
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -25,52 +25,61 @@ import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.GroupFrameAccessor;
 import edu.uci.ics.hyracks.dataflow.std.util.ReferenceEntry;
 import edu.uci.ics.hyracks.dataflow.std.util.ReferencedPriorityQueue;
 
 public class RunMergingFrameReader implements IFrameReader {
     private final IHyracksTaskContext ctx;
-    private final IFrameReader[] runCursors;
-    private final List<ByteBuffer> inFrames;
+    private final List<? extends IFrameReader> runCursors;
+    private final List<? extends IFrame> inFrames;
     private final int[] sortFields;
     private final IBinaryComparator[] comparators;
     private final INormalizedKeyComputer nmkComputer;
     private final RecordDescriptor recordDesc;
-    private final FrameTupleAppender outFrameAppender;
+    private final int topK;
+    private int tupleCount;
+    private FrameTupleAppender outFrameAppender;
     private ReferencedPriorityQueue topTuples;
     private int[] tupleIndexes;
-    private FrameTupleAccessor[] tupleAccessors;
+    private IFrameTupleAccessor[] tupleAccessors;
 
-    public RunMergingFrameReader(IHyracksTaskContext ctx, IFrameReader[] runCursors, List<ByteBuffer> inFrames,
-            int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
-            RecordDescriptor recordDesc) {
+    public RunMergingFrameReader(IHyracksTaskContext ctx, List<? extends IFrameReader> runs,
+            List<? extends IFrame> inFrames, int[] sortFields, IBinaryComparator[] comparators,
+            INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc) {
+        this(ctx, runs, inFrames, sortFields, comparators, nmkComputer, recordDesc, Integer.MAX_VALUE);
+    }
+
+    public RunMergingFrameReader(IHyracksTaskContext ctx, List<? extends IFrameReader> runs,
+            List<? extends IFrame> inFrames, int[] sortFields, IBinaryComparator[] comparators,
+            INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc, int topK) {
         this.ctx = ctx;
-        this.runCursors = runCursors;
+        this.runCursors = runs;
         this.inFrames = inFrames;
         this.sortFields = sortFields;
         this.comparators = comparators;
         this.nmkComputer = nmkComputer;
         this.recordDesc = recordDesc;
-        outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
+        this.topK = topK;
     }
 
     @Override
     public void open() throws HyracksDataException {
-        tupleAccessors = new FrameTupleAccessor[runCursors.length];
+        tupleCount = 0;
+        tupleAccessors = new IFrameTupleAccessor[runCursors.size()];
+        outFrameAppender = new FrameTupleAppender();
         Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
-        topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, runCursors.length, comparator,
-                sortFields, nmkComputer);
-        tupleIndexes = new int[runCursors.length];
-        for (int i = 0; i < runCursors.length; i++) {
+        topTuples = new ReferencedPriorityQueue(runCursors.size(), comparator, sortFields, nmkComputer);
+        tupleIndexes = new int[runCursors.size()];
+        for (int i = 0; i < runCursors.size(); i++) {
             tupleIndexes[i] = 0;
             int runIndex = topTuples.peek().getRunid();
-            runCursors[runIndex].open();
-            if (runCursors[runIndex].nextFrame(inFrames.get(runIndex))) {
-                tupleAccessors[runIndex] = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
-                tupleAccessors[runIndex].reset(inFrames.get(runIndex));
-                setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+            runCursors.get(runIndex).open();
+            if (runCursors.get(runIndex).nextFrame(inFrames.get(runIndex))) {
+                tupleAccessors[runIndex] = new GroupFrameAccessor(ctx.getInitialFrameSize(), recordDesc);
+                tupleAccessors[runIndex].reset(inFrames.get(runIndex).getBuffer());
+                setNextTopTuple(runIndex, tupleIndexes, runCursors, inFrames, tupleAccessors, topTuples);
             } else {
                 closeRun(runIndex, runCursors, tupleAccessors);
                 topTuples.pop();
@@ -79,20 +88,21 @@ public class RunMergingFrameReader implements IFrameReader {
     }
 
     @Override
-    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        outFrameAppender.reset(buffer, true);
-        while (!topTuples.areRunsExhausted()) {
+    public boolean nextFrame(IFrame outFrame) throws HyracksDataException {
+        outFrameAppender.reset(outFrame, true);
+        while (!topTuples.areRunsExhausted() && tupleCount < topK) {
             ReferenceEntry top = topTuples.peek();
             int runIndex = top.getRunid();
-            FrameTupleAccessor fta = top.getAccessor();
+            IFrameTupleAccessor fta = top.getAccessor();
             int tupleIndex = top.getTupleIndex();
 
             if (!outFrameAppender.append(fta, tupleIndex)) {
                 return true;
+            } else {
+                tupleCount++;
             }
-
             ++tupleIndexes[runIndex];
-            setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+            setNextTopTuple(runIndex, tupleIndexes, runCursors, inFrames, tupleAccessors, topTuples);
         }
 
         if (outFrameAppender.getTupleCount() > 0) {
@@ -103,14 +113,15 @@ public class RunMergingFrameReader implements IFrameReader {
 
     @Override
     public void close() throws HyracksDataException {
-        for (int i = 0; i < runCursors.length; ++i) {
+        for (int i = 0; i < runCursors.size(); ++i) {
             closeRun(i, runCursors, tupleAccessors);
         }
     }
 
-    private void setNextTopTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
-            FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws HyracksDataException {
-        boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+    private static void setNextTopTuple(int runIndex, int[] tupleIndexes, List<? extends IFrameReader> runCursors,
+            List<? extends IFrame> inFrames, IFrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples)
+            throws HyracksDataException {
+        boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, inFrames, tupleAccessors);
         if (exists) {
             topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
         } else {
@@ -119,15 +130,16 @@ public class RunMergingFrameReader implements IFrameReader {
         }
     }
 
-    private boolean hasNextTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
-            FrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
-        if (tupleAccessors[runIndex] == null || runCursors[runIndex] == null) {
+    private static boolean hasNextTuple(int runIndex, int[] tupleIndexes, List<? extends IFrameReader> runCursors,
+            List<? extends IFrame> inFrames, IFrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
+        if (tupleAccessors[runIndex] == null || runCursors.get(runIndex) == null) {
             return false;
         } else if (tupleIndexes[runIndex] >= tupleAccessors[runIndex].getTupleCount()) {
-            ByteBuffer buf = tupleAccessors[runIndex].getBuffer(); // same-as-inFrames.get(runIndex)
-            if (runCursors[runIndex].nextFrame(buf)) {
+            IFrame frame = inFrames.get(runIndex);
+            if (runCursors.get(runIndex).nextFrame(frame)) {
                 tupleIndexes[runIndex] = 0;
-                return hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+                tupleAccessors[runIndex].reset(frame.getBuffer());
+                return hasNextTuple(runIndex, tupleIndexes, runCursors, inFrames, tupleAccessors);
             } else {
                 return false;
             }
@@ -136,11 +148,12 @@ public class RunMergingFrameReader implements IFrameReader {
         }
     }
 
-    private void closeRun(int index, IFrameReader[] runCursors, IFrameTupleAccessor[] tupleAccessors)
+    private static void closeRun(int index, List<? extends IFrameReader> runCursors,
+            IFrameTupleAccessor[] tupleAccessors)
             throws HyracksDataException {
-        if (runCursors[index] != null) {
-            runCursors[index].close();
-            runCursors[index] = null;
+        if (runCursors.get(index) != null) {
+            runCursors.get(index).close();
+            runCursors.set(index, null);
             tupleAccessors[index] = null;
         }
     }
@@ -153,8 +166,8 @@ public class RunMergingFrameReader implements IFrameReader {
                 if (nmk1 != nmk2) {
                     return ((((long) nmk1) & 0xffffffffL) < (((long) nmk2) & 0xffffffffL)) ? -1 : 1;
                 }
-                FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
-                FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
+                IFrameTupleAccessor fta1 = tp1.getAccessor();
+                IFrameTupleAccessor fta2 = tp2.getAccessor();
                 byte[] b1 = fta1.getBuffer().array();
                 byte[] b2 = fta2.getBuffer().array();
                 int[] tPointers1 = tp1.getTPointers();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Slot.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Slot.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Slot.java
deleted file mode 100644
index 73f99dd..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Slot.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-/**
- * @author pouria
- *         Defines a slot in the memory, which can be a free or used (allocated)
- *         slot. Memory is a set of frames, ordered as a list. Each tuple is
- *         stored in a slot, where the location of the slot is denoted by a pair
- *         of integers:
- *         - The index of the frame, in the list of frames in memory. (referred
- *         to as frameIx)
- *         - The starting offset of the slot, within that specific frame.
- *         (referred to as offset)
- */
-public class Slot {
-
-    private int frameIx;
-    private int offset;
-
-    public Slot() {
-        this.frameIx = BSTNodeUtil.INVALID_INDEX;
-        this.offset = BSTNodeUtil.INVALID_INDEX;
-    }
-
-    public Slot(int frameIx, int offset) {
-        this.frameIx = frameIx;
-        this.offset = offset;
-    }
-
-    public void set(int frameIx, int offset) {
-        this.frameIx = frameIx;
-        this.offset = offset;
-    }
-
-    public int getFrameIx() {
-        return frameIx;
-    }
-
-    public void setFrameIx(int frameIx) {
-        this.frameIx = frameIx;
-    }
-
-    public int getOffset() {
-        return offset;
-    }
-
-    public void setOffset(int offset) {
-        this.offset = offset;
-    }
-
-    public boolean isNull() {
-        return (frameIx == BSTNodeUtil.INVALID_INDEX) || (offset == BSTNodeUtil.INVALID_INDEX);
-    }
-
-    public void clear() {
-        this.frameIx = BSTNodeUtil.INVALID_INDEX;
-        this.offset = BSTNodeUtil.INVALID_INDEX;
-    }
-
-    public void copy(Slot s) {
-        this.frameIx = s.getFrameIx();
-        this.offset = s.getOffset();
-    }
-
-    public String toString() {
-        return "(" + frameIx + ", " + offset + ")";
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinHeap.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinHeap.java
deleted file mode 100644
index 1cde75f..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinHeap.java
+++ /dev/null
@@ -1,293 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-/**
- * @author pouria
- *         Implements a minimum binary heap, used as selection tree, for sort
- *         with replacement. This heap structure can only be used as the min
- *         heap (no access to the max element). Elements in the heap are
- *         compared based on their run numbers, and sorting key(s):
- *         Considering two heap elements A and B:
- *         if RunNumber(A) > RunNumber(B) then A is larger than B if
- *         RunNumber(A) == RunNumber(B), then A is smaller than B, if and only
- *         if the value of the sort key(s) in B is greater than A (based on the
- *         sort comparator).
- */
-public class SortMinHeap implements ISelectionTree {
-
-    static final int RUN_ID_IX = 0;
-    static final int FRAME_IX = 1;
-    static final int OFFSET_IX = 2;
-    private static final int PNK_IX = 3;
-    private static final int ELEMENT_SIZE = 4;
-    private static final int INIT_ARRAY_SIZE = 512;
-
-    private final int[] sortFields;
-    private final IBinaryComparator[] comparators;
-    private final RecordDescriptor recordDescriptor;
-    private final FrameTupleAccessor fta1;
-    private final FrameTupleAccessor fta2;
-    private int[] elements;
-    private int nextIx;
-    private final IMemoryManager memMgr;
-    private int[] top; // Used as a temp variable to access the top, to avoid object creation
-
-    public SortMinHeap(IHyracksCommonContext ctx, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDesc, IMemoryManager memMgr) {
-        this.sortFields = sortFields;
-        this.comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            this.comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
-        this.recordDescriptor = recordDesc;
-        fta1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        this.memMgr = memMgr;
-        this.top = new int[ELEMENT_SIZE];
-        Arrays.fill(top, -1);
-        this.elements = new int[INIT_ARRAY_SIZE];
-        Arrays.fill(elements, -1);
-        this.nextIx = 0;
-    }
-
-    /*
-     * Assumption (element structure): [RunId][FrameIx][Offset][Poorman NK]
-     */
-    @Override
-    public void getMin(int[] result) throws HyracksDataException {
-        if (nextIx == 0) {
-            result[0] = result[1] = result[2] = result[3] = -1;
-            return;
-        }
-
-        top = delete(0);
-        for (int i = 0; i < top.length; i++) {
-            result[i] = top[i];
-        }
-    }
-
-    @Override
-    public void peekMin(int[] result) {
-        if (nextIx == 0) {
-            result[0] = result[1] = result[2] = result[3] = -1;
-            return;
-        }
-        for (int i = 0; i < ELEMENT_SIZE; i++) {
-            result[i] = elements[i];
-        }
-    }
-
-    @Override
-    public void insert(int[] e) throws HyracksDataException {
-        if (nextIx >= elements.length) {
-            elements = Arrays.copyOf(elements, elements.length * 2);
-        }
-        for (int i = 0; i < ELEMENT_SIZE; i++) {
-            elements[nextIx + i] = e[i];
-        }
-        siftUp(nextIx);
-        nextIx += ELEMENT_SIZE;
-
-    }
-
-    @Override
-    public void reset() {
-        Arrays.fill(elements, -1);
-        nextIx = 0;
-    }
-
-    @Override
-    public boolean isEmpty() {
-        return (nextIx < ELEMENT_SIZE);
-    }
-
-    public int _debugGetSize() {
-        return (nextIx > 0 ? (nextIx - 1) / 4 : 0);
-    }
-
-    private int[] delete(int nix) throws HyracksDataException {
-        int[] nv = Arrays.copyOfRange(elements, nix, nix + ELEMENT_SIZE);
-        int[] lastElem = removeLast();
-
-        if (nextIx == 0) {
-            return nv;
-        }
-
-        for (int i = 0; i < ELEMENT_SIZE; i++) {
-            elements[nix + i] = lastElem[i];
-        }
-        int pIx = getParent(nix);
-        if (pIx > -1 && (compare(lastElem, Arrays.copyOfRange(elements, pIx, pIx + ELEMENT_SIZE)) < 0)) {
-            siftUp(nix);
-        } else {
-            siftDown(nix);
-        }
-        return nv;
-    }
-
-    private int[] removeLast() {
-        if (nextIx < ELEMENT_SIZE) { //this is the very last element
-            return new int[] { -1, -1, -1, -1 };
-        }
-        int[] l = Arrays.copyOfRange(elements, nextIx - ELEMENT_SIZE, nextIx);
-        Arrays.fill(elements, nextIx - ELEMENT_SIZE, nextIx, -1);
-        nextIx -= ELEMENT_SIZE;
-        return l;
-    }
-
-    private void siftUp(int nodeIx) throws HyracksDataException {
-        int p = getParent(nodeIx);
-        if (p < 0) {
-            return;
-        }
-        while (p > -1 && (compare(nodeIx, p) < 0)) {
-            swap(p, nodeIx);
-            nodeIx = p;
-            p = getParent(nodeIx);
-            if (p < 0) { // We are at the root
-                return;
-            }
-        }
-    }
-
-    private void siftDown(int nodeIx) throws HyracksDataException {
-        int mix = getMinOfChildren(nodeIx);
-        if (mix < 0) {
-            return;
-        }
-        while (mix > -1 && (compare(mix, nodeIx) < 0)) {
-            swap(mix, nodeIx);
-            nodeIx = mix;
-            mix = getMinOfChildren(nodeIx);
-            if (mix < 0) { // We hit the leaf level
-                return;
-            }
-        }
-    }
-
-    // first < sec : -1
-    private int compare(int nodeSIx1, int nodeSIx2) throws HyracksDataException {
-        int[] n1 = Arrays.copyOfRange(elements, nodeSIx1, nodeSIx1 + ELEMENT_SIZE);
-        int[] n2 = Arrays.copyOfRange(elements, nodeSIx2, nodeSIx2 + ELEMENT_SIZE);
-        return (compare(n1, n2));
-    }
-
-    // first < sec : -1
-    private int compare(int[] n1, int[] n2) throws HyracksDataException {
-        // Compare Run Numbers
-        if (n1[RUN_ID_IX] != n2[RUN_ID_IX]) {
-            return (n1[RUN_ID_IX] < n2[RUN_ID_IX] ? -1 : 1);
-        }
-
-        // Compare Poor man Normalized Keys
-        if (n1[PNK_IX] != n2[PNK_IX]) {
-            return ((((long) n1[PNK_IX]) & 0xffffffffL) < (((long) n2[PNK_IX]) & 0xffffffffL)) ? -1 : 1;
-        }
-
-        return compare(getFrame(n1[FRAME_IX]), getFrame(n2[FRAME_IX]), n1[OFFSET_IX], n2[OFFSET_IX]);
-    }
-
-    private int compare(ByteBuffer fr1, ByteBuffer fr2, int r1StartOffset, int r2StartOffset)
-            throws HyracksDataException {
-        byte[] b1 = fr1.array();
-        byte[] b2 = fr2.array();
-        fta1.reset(fr1);
-        fta2.reset(fr2);
-        int headerLen = BSTNodeUtil.HEADER_SIZE;
-        r1StartOffset += headerLen;
-        r2StartOffset += headerLen;
-        for (int f = 0; f < comparators.length; ++f) {
-            int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : fr1.getInt(r1StartOffset + (fIdx - 1) * 4);
-            int f1End = fr1.getInt(r1StartOffset + fIdx * 4);
-            int s1 = r1StartOffset + fta1.getFieldSlotsLength() + f1Start;
-            int l1 = f1End - f1Start;
-            int f2Start = fIdx == 0 ? 0 : fr2.getInt(r2StartOffset + (fIdx - 1) * 4);
-            int f2End = fr2.getInt(r2StartOffset + fIdx * 4);
-            int s2 = r2StartOffset + fta2.getFieldSlotsLength() + f2Start;
-            int l2 = f2End - f2Start;
-
-            int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
-
-            if (c != 0) {
-                return c;
-            }
-        }
-        return 0;
-    }
-
-    private int getMinOfChildren(int nix) throws HyracksDataException { // returns index of min child
-        int lix = getLeftChild(nix);
-        if (lix < 0) {
-            return -1;
-        }
-        int rix = getRightChild(nix);
-        if (rix < 0) {
-            return lix;
-        }
-        return ((compare(lix, rix) < 0) ? lix : rix);
-    }
-
-    //Assumption: n1Ix and n2Ix are starting indices of two elements
-    private void swap(int n1Ix, int n2Ix) {
-        int[] temp = Arrays.copyOfRange(elements, n1Ix, n1Ix + ELEMENT_SIZE);
-        for (int i = 0; i < ELEMENT_SIZE; i++) {
-            elements[n1Ix + i] = elements[n2Ix + i];
-            elements[n2Ix + i] = temp[i];
-        }
-    }
-
-    private int getLeftChild(int ix) {
-        int lix = (2 * ELEMENT_SIZE) * (ix / ELEMENT_SIZE) + ELEMENT_SIZE;
-        return ((lix < nextIx) ? lix : -1);
-    }
-
-    private int getRightChild(int ix) {
-        int rix = (2 * ELEMENT_SIZE) * (ix / ELEMENT_SIZE) + (2 * ELEMENT_SIZE);
-        return ((rix < nextIx) ? rix : -1);
-    }
-
-    private int getParent(int ix) {
-        if (ix <= 0) {
-            return -1;
-        }
-        return ((ix - ELEMENT_SIZE) / (2 * ELEMENT_SIZE)) * ELEMENT_SIZE;
-    }
-
-    private ByteBuffer getFrame(int frameIx) {
-        return (memMgr.getFrame(frameIx));
-    }
-
-    @Override
-    public void getMax(int[] result) {
-        throw new IllegalStateException("getMax() method not applicable to Min Heap");
-    }
-
-    @Override
-    public void peekMax(int[] result) {
-        throw new IllegalStateException("getMax() method not applicable to Min Heap");
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinMaxHeap.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinMaxHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinMaxHeap.java
deleted file mode 100644
index 12aa8a1..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinMaxHeap.java
+++ /dev/null
@@ -1,448 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-/**
- * @author pouria
- *         Implements a MinMax binary heap, used as the selection tree, in
- *         sorting with replacement. Check SortMinHeap for details on comparing
- *         elements.
- */
-public class SortMinMaxHeap implements ISelectionTree {
-    static final int RUN_ID_IX = 0;
-    static final int FRAME_IX = 1;
-    static final int OFFSET_IX = 2;
-    private static final int PNK_IX = 3;
-    private static final int NOT_EXIST = -1;
-    private static final int ELEMENT_SIZE = 4;
-    private static final int INIT_ARRAY_SIZE = 512;
-
-    private final int[] sortFields;
-    private final IBinaryComparator[] comparators;
-    private final RecordDescriptor recordDescriptor;
-    private final FrameTupleAccessor fta1;
-    private final FrameTupleAccessor fta2;
-
-    private int[] elements;
-    private int nextIx;
-
-    private final IMemoryManager memMgr;
-
-    public SortMinMaxHeap(IHyracksCommonContext ctx, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
-            RecordDescriptor recordDesc, IMemoryManager memMgr) {
-        this.sortFields = sortFields;
-        this.comparators = new IBinaryComparator[comparatorFactories.length];
-        for (int i = 0; i < comparatorFactories.length; ++i) {
-            this.comparators[i] = comparatorFactories[i].createBinaryComparator();
-        }
-        this.recordDescriptor = recordDesc;
-        fta1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        this.memMgr = memMgr;
-        this.elements = new int[INIT_ARRAY_SIZE];
-        Arrays.fill(elements, -1);
-        this.nextIx = 0;
-    }
-
-    @Override
-    public void insert(int[] element) throws HyracksDataException {
-        if (nextIx >= elements.length) {
-            elements = Arrays.copyOf(elements, elements.length * 2);
-        }
-        for (int i = 0; i < ELEMENT_SIZE; i++) {
-            elements[nextIx + i] = element[i];
-        }
-        nextIx += ELEMENT_SIZE;
-        bubbleUp(nextIx - ELEMENT_SIZE);
-    }
-
-    @Override
-    public void getMin(int[] result) throws HyracksDataException {
-        if (nextIx == 0) {
-            result[0] = result[1] = result[2] = result[3] = -1;
-            return;
-        }
-
-        int[] topElem = delete(0);
-        for (int x = 0; x < ELEMENT_SIZE; x++) {
-            result[x] = topElem[x];
-        }
-    }
-
-    @Override
-    public void reset() {
-        Arrays.fill(elements, -1);
-        nextIx = 0;
-    }
-
-    @Override
-    public boolean isEmpty() {
-        return (nextIx < ELEMENT_SIZE);
-    }
-
-    @Override
-    public void peekMin(int[] result) {
-        if (nextIx == 0) {
-            result[0] = result[1] = result[2] = result[3] = -1;
-            return;
-        }
-
-        for (int x = 0; x < ELEMENT_SIZE; x++) {
-            result[x] = elements[x];
-        }
-    }
-
-    @Override
-    public void getMax(int[] result) throws HyracksDataException {
-        if (nextIx == ELEMENT_SIZE) {
-            int[] topElement = removeLast();
-            for (int x = 0; x < ELEMENT_SIZE; x++) {
-                result[x] = topElement[x];
-            }
-            return;
-        }
-
-        if (nextIx > ELEMENT_SIZE) {
-            int lc = getLeftChild(0);
-            int rc = getRightChild(0);
-            int maxIx = lc;
-
-            if (rc != -1) {
-                maxIx = compare(lc, rc) < 0 ? rc : lc;
-            }
-
-            int[] maxElem = delete(maxIx);
-            for (int x = 0; x < ELEMENT_SIZE; x++) {
-                result[x] = maxElem[x];
-            }
-            return;
-        }
-
-        result[0] = result[1] = result[2] = result[3] = -1;
-
-    }
-
-    @Override
-    public void peekMax(int[] result) throws HyracksDataException {
-        if (nextIx == ELEMENT_SIZE) {
-            for (int i = 0; i < ELEMENT_SIZE; i++) {
-                result[i] = elements[i];
-            }
-            return;
-        }
-        if (nextIx > ELEMENT_SIZE) {
-            int lc = getLeftChild(0);
-            int rc = getRightChild(0);
-            int maxIx = lc;
-
-            if (rc != -1) {
-                maxIx = compare(lc, rc) < 0 ? rc : lc;
-            }
-
-            for (int x = 0; x < ELEMENT_SIZE; x++) {
-                result[x] = elements[maxIx + x];
-            }
-
-            return;
-        }
-        result[0] = result[1] = result[2] = result[3] = -1;
-    }
-
-    private int[] delete(int delIx) throws HyracksDataException {
-        int s = nextIx;
-        if (nextIx > ELEMENT_SIZE) {
-            int[] delEntry = Arrays.copyOfRange(elements, delIx, delIx + ELEMENT_SIZE);
-            int[] last = removeLast();
-            if (delIx != (s - ELEMENT_SIZE)) {
-                for (int x = 0; x < ELEMENT_SIZE; x++) {
-                    elements[delIx + x] = last[x];
-                }
-                trickleDown(delIx);
-            }
-            return delEntry;
-        } else if (nextIx == ELEMENT_SIZE) {
-            return (removeLast());
-        }
-        return null;
-    }
-
-    private int[] removeLast() {
-        if (nextIx < ELEMENT_SIZE) { //this is the very last element
-            return new int[] { -1, -1, -1, -1 };
-        }
-        int[] l = Arrays.copyOfRange(elements, nextIx - ELEMENT_SIZE, nextIx);
-        Arrays.fill(elements, nextIx - ELEMENT_SIZE, nextIx, -1);
-        nextIx -= ELEMENT_SIZE;
-        return l;
-    }
-
-    private void bubbleUp(int ix) throws HyracksDataException {
-        int p = getParentIx(ix);
-        if (isAtMinLevel(ix)) {
-            if (p != NOT_EXIST && compare(p, ix) < 0) {
-                swap(ix, p);
-                bubbleUpMax(p);
-            } else {
-                bubbleUpMin(ix);
-            }
-        } else { // i is at max level
-            if (p != NOT_EXIST && compare(ix, p) < 0) {
-                swap(ix, p);
-                bubbleUpMin(p);
-            } else {
-                bubbleUpMax(ix);
-            }
-        }
-    }
-
-    private void bubbleUpMax(int ix) throws HyracksDataException {
-        int gp = getGrandParent(ix);
-        if (gp != NOT_EXIST && compare(gp, ix) < 0) {
-            swap(ix, gp);
-            bubbleUpMax(gp);
-        }
-    }
-
-    private void bubbleUpMin(int ix) throws HyracksDataException {
-        int gp = getGrandParent(ix);
-        if (gp != NOT_EXIST && compare(ix, gp) < 0) {
-            swap(ix, gp);
-            bubbleUpMin(gp);
-        }
-    }
-
-    private void trickleDown(int ix) throws HyracksDataException {
-        if (isAtMinLevel(ix)) {
-            trickleDownMin(ix);
-        } else {
-            trickleDownMax(ix);
-        }
-    }
-
-    private void trickleDownMax(int ix) throws HyracksDataException {
-        int maxIx = getMaxOfDescendents(ix);
-        if (maxIx == NOT_EXIST) {
-            return;
-        }
-        if (maxIx > getLeftChild(ix) && maxIx > getRightChild(ix)) { // A grand
-                                                                     // children
-            if (compare(ix, maxIx) < 0) {
-                swap(maxIx, ix);
-                int p = getParentIx(maxIx);
-                if (p != NOT_EXIST && compare(maxIx, p) < 0) {
-                    swap(maxIx, p);
-                }
-                trickleDownMax(maxIx);
-            }
-        } else { // A children
-            if (compare(ix, maxIx) < 0) {
-                swap(ix, maxIx);
-            }
-        }
-    }
-
-    private void trickleDownMin(int ix) throws HyracksDataException {
-        int minIx = getMinOfDescendents(ix);
-        if (minIx == NOT_EXIST) {
-            return;
-        }
-        if (minIx > getLeftChild(ix) && minIx > getRightChild(ix)) { // A grand
-                                                                     // children
-            if (compare(minIx, ix) < 0) {
-                swap(minIx, ix);
-                int p = getParentIx(minIx);
-                if (p != NOT_EXIST && compare(p, minIx) < 0) {
-                    swap(minIx, p);
-                }
-                trickleDownMin(minIx);
-            }
-        } else { // A children
-            if (compare(minIx, ix) < 0) {
-                swap(ix, minIx);
-            }
-        }
-    }
-
-    // Min among children and grand children
-    private int getMinOfDescendents(int ix) throws HyracksDataException {
-        int lc = getLeftChild(ix);
-        if (lc == NOT_EXIST) {
-            return NOT_EXIST;
-        }
-        int rc = getRightChild(ix);
-        if (rc == NOT_EXIST) {
-            return lc;
-        }
-        int min = (compare(lc, rc) < 0) ? lc : rc;
-        int[] lgc = getLeftGrandChildren(ix);
-        int[] rgc = getRightGrandChildren(ix);
-        for (int k = 0; k < 2; k++) {
-            if (lgc[k] != NOT_EXIST && compare(lgc[k], min) < 0) {
-                min = lgc[k];
-            }
-            if (rgc[k] != NOT_EXIST && compare(rgc[k], min) < 0) {
-                min = rgc[k];
-            }
-        }
-        return min;
-    }
-
-    // Max among children and grand children
-    private int getMaxOfDescendents(int ix) throws HyracksDataException {
-        int lc = getLeftChild(ix);
-        if (lc == NOT_EXIST) {
-            return NOT_EXIST;
-        }
-        int rc = getRightChild(ix);
-        if (rc == NOT_EXIST) {
-            return lc;
-        }
-        int max = (compare(lc, rc) < 0) ? rc : lc;
-        int[] lgc = getLeftGrandChildren(ix);
-        int[] rgc = getRightGrandChildren(ix);
-        for (int k = 0; k < 2; k++) {
-            if (lgc[k] != NOT_EXIST && compare(max, lgc[k]) < 0) {
-                max = lgc[k];
-            }
-            if (rgc[k] != NOT_EXIST && compare(max, rgc[k]) < 0) {
-                max = rgc[k];
-            }
-        }
-        return max;
-    }
-
-    private void swap(int n1Ix, int n2Ix) {
-        int[] temp = Arrays.copyOfRange(elements, n1Ix, n1Ix + ELEMENT_SIZE);
-        for (int i = 0; i < ELEMENT_SIZE; i++) {
-            elements[n1Ix + i] = elements[n2Ix + i];
-            elements[n2Ix + i] = temp[i];
-        }
-    }
-
-    private int getParentIx(int i) {
-        if (i < ELEMENT_SIZE) {
-            return NOT_EXIST;
-        }
-        return ((i - ELEMENT_SIZE) / (2 * ELEMENT_SIZE)) * ELEMENT_SIZE;
-    }
-
-    private int getGrandParent(int i) {
-        int p = getParentIx(i);
-        return p != -1 ? getParentIx(p) : NOT_EXIST;
-    }
-
-    private int getLeftChild(int i) {
-        int lc = (2 * ELEMENT_SIZE) * (i / ELEMENT_SIZE) + ELEMENT_SIZE;
-        return (lc < nextIx ? lc : -1);
-    }
-
-    private int[] getLeftGrandChildren(int i) {
-        int lc = getLeftChild(i);
-        return lc != NOT_EXIST ? new int[] { getLeftChild(lc), getRightChild(lc) } : new int[] { NOT_EXIST, NOT_EXIST };
-    }
-
-    private int getRightChild(int i) {
-        int rc = (2 * ELEMENT_SIZE) * (i / ELEMENT_SIZE) + (2 * ELEMENT_SIZE);
-        return (rc < nextIx ? rc : -1);
-    }
-
-    private int[] getRightGrandChildren(int i) {
-        int rc = getRightChild(i);
-        return rc != NOT_EXIST ? new int[] { getLeftChild(rc), getRightChild(rc) } : new int[] { NOT_EXIST, NOT_EXIST };
-    }
-
-    private boolean isAtMinLevel(int i) {
-        int l = getLevel(i);
-        return l % 2 == 0 ? true : false;
-    }
-
-    private int getLevel(int i) {
-        if (i < ELEMENT_SIZE) {
-            return 0;
-        }
-
-        int cnv = i / ELEMENT_SIZE;
-        int l = (int) Math.floor(Math.log(cnv) / Math.log(2));
-        if (cnv == (((int) Math.pow(2, (l + 1))) - 1)) {
-            return (l + 1);
-        }
-        return l;
-    }
-
-    private ByteBuffer getFrame(int frameIx) {
-        return (memMgr.getFrame(frameIx));
-    }
-
-    // first < sec : -1
-    private int compare(int nodeSIx1, int nodeSIx2) throws HyracksDataException {
-        int[] n1 = Arrays.copyOfRange(elements, nodeSIx1, nodeSIx1 + ELEMENT_SIZE); //tree.get(nodeSIx1);
-        int[] n2 = Arrays.copyOfRange(elements, nodeSIx2, nodeSIx2 + ELEMENT_SIZE); //tree.get(nodeSIx2);
-        return (compare(n1, n2));
-    }
-
-    // first < sec : -1
-    private int compare(int[] n1, int[] n2) throws HyracksDataException {
-        // Compare Run Numbers
-        if (n1[RUN_ID_IX] != n2[RUN_ID_IX]) {
-            return (n1[RUN_ID_IX] < n2[RUN_ID_IX] ? -1 : 1);
-        }
-
-        // Compare Poor man Normalized Keys
-        if (n1[PNK_IX] != n2[PNK_IX]) {
-            return ((((long) n1[PNK_IX]) & 0xffffffffL) < (((long) n2[PNK_IX]) & 0xffffffffL)) ? -1 : 1;
-        }
-
-        return compare(getFrame(n1[FRAME_IX]), getFrame(n2[FRAME_IX]), n1[OFFSET_IX], n2[OFFSET_IX]);
-    }
-
-    private int compare(ByteBuffer fr1, ByteBuffer fr2, int r1StartOffset, int r2StartOffset)
-            throws HyracksDataException {
-        byte[] b1 = fr1.array();
-        byte[] b2 = fr2.array();
-        fta1.reset(fr1);
-        fta2.reset(fr2);
-        int headerLen = BSTNodeUtil.HEADER_SIZE;
-        r1StartOffset += headerLen;
-        r2StartOffset += headerLen;
-        for (int f = 0; f < comparators.length; ++f) {
-            int fIdx = sortFields[f];
-            int f1Start = fIdx == 0 ? 0 : fr1.getInt(r1StartOffset + (fIdx - 1) * 4);
-            int f1End = fr1.getInt(r1StartOffset + fIdx * 4);
-            int s1 = r1StartOffset + fta1.getFieldSlotsLength() + f1Start;
-            int l1 = f1End - f1Start;
-            int f2Start = fIdx == 0 ? 0 : fr2.getInt(r2StartOffset + (fIdx - 1) * 4);
-            int f2End = fr2.getInt(r2StartOffset + fIdx * 4);
-            int s2 = r2StartOffset + fta2.getFieldSlotsLength() + f2Start;
-            int l2 = f2End - f2Start;
-
-            int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
-
-            if (c != 0) {
-                return c;
-            }
-        }
-        return 0;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
new file mode 100644
index 0000000..ee43993
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.value.*;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+
+import java.util.List;
+
+public class TopKSorterOperatorDescriptor extends AbstractSorterOperatorDescriptor {
+
+    private final int topK;
+
+    public TopKSorterOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int topK, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor) {
+        super(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
+        this.topK = topK;
+    }
+
+    @Override
+    public SortActivity getSortActivity(ActivityId id) {
+        return new SortActivity(id) {
+            @Override
+            protected AbstractSortRunGenerator getRunGenerator(IHyracksTaskContext ctx,
+                    IRecordDescriptorProvider recordDescProvider) {
+                return new HybridTopKSortRunGenerator(ctx, framesLimit, topK, sortFields, firstKeyNormalizerFactory,
+                        comparatorFactories, recordDescriptors[0]);
+
+            }
+        };
+    }
+
+    @Override
+    public MergeActivity getMergeActivity(ActivityId id) {
+        return new MergeActivity(id) {
+            @Override
+            protected ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
+                    IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter, List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators,
+                    INormalizedKeyComputer nmkComputer, int necessaryFrames) {
+                return new ExternalSortRunMerger(ctx, sorter, runs, sortFields, comparators,
+                        nmkComputer, recordDescriptors[0], necessaryFrames, topK, writer);
+            }
+        };
+    }
+}


[09/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 506da2e..ed25b4f 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -19,7 +19,9 @@ import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
@@ -74,7 +76,7 @@ public class OptimizedHybridHashJoin {
     private final boolean isLeftOuter;
     private final INullWriter[] nullWriters1;
 
-    private ByteBuffer[] memBuffs; //Memory buffers for build
+    private IFrame[] memBuffs; //Memory buffers for build
     private int[] curPBuff; //Current (last) Buffer for each partition
     private int[] nextBuff; //Next buffer in the partition's buffer chain
     private int[] buildPSizeInTups; //Size of build partitions (in tuples)
@@ -92,9 +94,9 @@ public class OptimizedHybridHashJoin {
     private FrameTupleAppender probeTupAppenderToSpilled;
 
     private int numOfSpilledParts;
-    private ByteBuffer[] sPartBuffs; //Buffers for probe spilled partitions (one buffer per spilled partition)
-    private ByteBuffer probeResBuff; //Buffer for probe resident partition tuples
-    private ByteBuffer reloadBuffer; //Buffer for reloading spilled partitions during partition tuning 
+    private IFrame[] sPartBuffs; //Buffers for probe spilled partitions (one buffer per spilled partition)
+    private IFrame probeResBuff; //Buffer for probe resident partition tuples
+    private IFrame reloadBuffer; //Buffer for reloading spilled partitions during partition tuning
 
     private int[] buildPSizeInFrames; //Used for partition tuning
     private int freeFramesCounter; //Used for partition tuning
@@ -124,8 +126,8 @@ public class OptimizedHybridHashJoin {
         this.buildRFWriters = new RunFileWriter[numOfPartitions];
         this.probeRFWriters = new RunFileWriter[numOfPartitions];
 
-        this.accessorBuild = new FrameTupleAccessor(ctx.getFrameSize(), buildRd);
-        this.accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), probeRd);
+        this.accessorBuild = new FrameTupleAccessor(buildRd);
+        this.accessorProbe = new FrameTupleAccessor(probeRd);
 
         this.predEvaluator = predEval;
         this.isLeftOuter = false;
@@ -154,8 +156,8 @@ public class OptimizedHybridHashJoin {
         this.buildRFWriters = new RunFileWriter[numOfPartitions];
         this.probeRFWriters = new RunFileWriter[numOfPartitions];
 
-        this.accessorBuild = new FrameTupleAccessor(ctx.getFrameSize(), buildRd);
-        this.accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), probeRd);
+        this.accessorBuild = new FrameTupleAccessor(buildRd);
+        this.accessorProbe = new FrameTupleAccessor(probeRd);
 
         this.predEvaluator = predEval;
         this.isLeftOuter = isLeftOuter;
@@ -170,7 +172,7 @@ public class OptimizedHybridHashJoin {
     }
 
     public void initBuild() throws HyracksDataException {
-        memBuffs = new ByteBuffer[memForJoin];
+        memBuffs = new IFrame[memForJoin];
         curPBuff = new int[numOfPartitions];
         nextBuff = new int[memForJoin];
         pStatus = new BitSet(numOfPartitions);
@@ -179,19 +181,22 @@ public class OptimizedHybridHashJoin {
         buildPSizeInFrames = new int[numOfPartitions];
         freeFramesCounter = memForJoin - numOfPartitions;
 
-        for (int i = 0; i < numOfPartitions; i++) { //Allocating one buffer per partition and setting as the head of the chain of buffers for that partition
-            memBuffs[i] = ctx.allocateFrame();
+        for (int i = 0; i
+                < numOfPartitions; i++) { //Allocating one buffer per partition and setting as the head of the chain of buffers for that partition
+            memBuffs[i] = new VSizeFrame(ctx);
             curPBuff[i] = i;
             nextBuff[i] = -1;
             buildPSizeInFrames[i] = 1; //The dedicated initial buffer
         }
 
-        nextFreeBuffIx = ((numOfPartitions < memForJoin) ? numOfPartitions : NO_MORE_FREE_BUFFER); //Setting the chain of unallocated frames
+        nextFreeBuffIx = ((numOfPartitions < memForJoin) ?
+                numOfPartitions :
+                NO_MORE_FREE_BUFFER); //Setting the chain of unallocated frames
         for (int i = numOfPartitions; i < memBuffs.length; i++) {
             nextBuff[i] = UNALLOCATED_FRAME;
         }
 
-        buildTupAppender = new FrameTupleAppender(ctx.getFrameSize());
+        buildTupAppender = new FrameTupleAppender();
 
     }
 
@@ -213,7 +218,7 @@ public class OptimizedHybridHashJoin {
     }
 
     private void processTuple(int tid, int pid) throws HyracksDataException {
-        ByteBuffer partition = memBuffs[curPBuff[pid]]; //Getting current buffer for the target partition
+        IFrame partition = memBuffs[curPBuff[pid]]; //Getting current buffer for the target partition
 
         if (!pStatus.get(pid)) { //resident partition
             buildTupAppender.reset(partition, false);
@@ -226,7 +231,8 @@ public class OptimizedHybridHashJoin {
                 if (newBuffIx == NO_MORE_FREE_BUFFER) { //Spill one partition
                     int pidToSpill = selectPartitionToSpill();
                     if (pidToSpill == -1) { //No more partition to spill
-                        throw new HyracksDataException("not enough memory for Hash Join (Allocation exceeds the limit)");
+                        throw new HyracksDataException(
+                                "not enough memory for Hash Join (Allocation exceeds the limit)");
                     }
                     spillPartition(pidToSpill);
                     buildTupAppender.reset(memBuffs[pidToSpill], true);
@@ -249,8 +255,8 @@ public class OptimizedHybridHashJoin {
                     break;
                 }
                 //Dedicated in-memory buffer for the partition is full, needed to be flushed first 
-                buildWrite(pid, partition);
-                partition.clear();
+                buildWrite(pid, partition.getBuffer());
+                partition.reset();
                 needClear = true;
                 buildPSizeInFrames[pid]++;
             }
@@ -260,7 +266,7 @@ public class OptimizedHybridHashJoin {
     private int allocateFreeBuffer(int pid) throws HyracksDataException {
         if (nextFreeBuffIx != NO_MORE_FREE_BUFFER) {
             if (memBuffs[nextFreeBuffIx] == null) {
-                memBuffs[nextFreeBuffIx] = ctx.allocateFrame();
+                memBuffs[nextFreeBuffIx] = new VSizeFrame(ctx);
             }
             int curPartBuffIx = curPBuff[pid];
             curPBuff[pid] = nextFreeBuffIx;
@@ -274,7 +280,7 @@ public class OptimizedHybridHashJoin {
             } else {
                 nextFreeBuffIx = oldNext;
             }
-            (memBuffs[curPBuff[pid]]).clear();
+            memBuffs[curPBuff[pid]].reset();
 
             freeFramesCounter--;
             return (curPBuff[pid]);
@@ -300,11 +306,10 @@ public class OptimizedHybridHashJoin {
                 + " frames for Thread ID " + Thread.currentThread().getId() + " (free frames: " + freeFramesCounter
                 + ").");
         int curBuffIx = curPBuff[pid];
-        ByteBuffer buff = null;
         while (curBuffIx != END_OF_PARTITION) {
-            buff = memBuffs[curBuffIx];
-            buildWrite(pid, buff);
-            buff.clear();
+            IFrame frame = memBuffs[curBuffIx];
+            buildWrite(pid, frame.getBuffer());
+            frame.reset();
 
             int freedBuffIx = curBuffIx;
             curBuffIx = nextBuff[curBuffIx];
@@ -346,8 +351,9 @@ public class OptimizedHybridHashJoin {
         }
 
         ByteBuffer buff = null;
-        for (int i = pStatus.nextSetBit(0); i >= 0; i = pStatus.nextSetBit(i + 1)) { //flushing and DeAllocating the dedicated buffers for the spilled partitions
-            buff = memBuffs[i];
+        for (int i = pStatus.nextSetBit(0); i >= 0; i = pStatus
+                .nextSetBit(i + 1)) { //flushing and DeAllocating the dedicated buffers for the spilled partitions
+            buff = memBuffs[i].getBuffer();
             accessorBuild.reset(buff);
             if (accessorBuild.getTupleCount() > 0) {
                 buildWrite(i, buff);
@@ -389,7 +395,7 @@ public class OptimizedHybridHashJoin {
     }
 
     private void partitionTune() throws HyracksDataException {
-        reloadBuffer = ctx.allocateFrame();
+        reloadBuffer = new VSizeFrame(ctx);
         ArrayList<Integer> reloadSet = selectPartitionsToReload();
         for (int i = 0; i < reloadSet.size(); i++) {
             int pid = reloadSet.get(i);
@@ -414,7 +420,6 @@ public class OptimizedHybridHashJoin {
             loadPartitionInMem(pid, buildRFWriters[pid], buffsToLoad);
         }
         reloadSet.clear();
-        reloadSet = null;
     }
 
     private void loadPartitionInMem(int pid, RunFileWriter wr, int[] buffs) throws HyracksDataException {
@@ -422,16 +427,16 @@ public class OptimizedHybridHashJoin {
         r.open();
         int counter = 0;
         ByteBuffer mBuff = null;
-        reloadBuffer.clear();
+        reloadBuffer.reset();
         while (r.nextFrame(reloadBuffer)) {
-            mBuff = memBuffs[buffs[counter]];
-            if (mBuff == null) {
-                mBuff = ctx.allocateFrame();
-                memBuffs[buffs[counter]] = mBuff;
+            if (memBuffs[buffs[counter]] == null) {
+                memBuffs[buffs[counter]] = new VSizeFrame(ctx);
             }
-            FrameUtils.copy(reloadBuffer, mBuff);
+            memBuffs[buffs[counter]].ensureFrameSize(reloadBuffer.getFrameSize());
+            mBuff = memBuffs[buffs[counter]].getBuffer();
+            FrameUtils.copyAndFlip(reloadBuffer.getBuffer(), mBuff);
             counter++;
-            reloadBuffer.clear();
+            reloadBuffer.reset();
         }
 
         int curNext = nextBuff[buffs[buffs.length - 1]];
@@ -459,10 +464,10 @@ public class OptimizedHybridHashJoin {
 
     private void createInMemoryJoiner(int inMemTupCount) throws HyracksDataException {
         ISerializableTable table = new SerializableHashTable(inMemTupCount, ctx);
-        this.inMemJoiner = new InMemoryHashJoin(ctx, inMemTupCount,
-                new FrameTupleAccessor(ctx.getFrameSize(), probeRd), probeHpc, new FrameTupleAccessor(
-                        ctx.getFrameSize(), buildRd), buildHpc, new FrameTuplePairComparator(probeKeys, buildKeys,
-                        comparators), isLeftOuter, nullWriters1, table, predEvaluator, isReversed);
+        this.inMemJoiner = new InMemoryHashJoin(ctx, inMemTupCount, new FrameTupleAccessor(probeRd), probeHpc,
+                new FrameTupleAccessor(buildRd), buildHpc,
+                new FrameTuplePairComparator(probeKeys, buildKeys, comparators), isLeftOuter, nullWriters1, table,
+                predEvaluator, isReversed);
     }
 
     private void cacheInMemJoin() throws HyracksDataException {
@@ -471,7 +476,7 @@ public class OptimizedHybridHashJoin {
             if (!pStatus.get(pid)) {
                 int nextBuffIx = curPBuff[pid];
                 while (nextBuffIx > -1) { //It is not Invalid or End_Of_Partition
-                    inMemJoiner.build(memBuffs[nextBuffIx]);
+                    inMemJoiner.build(memBuffs[nextBuffIx].getBuffer());
                     nextBuffIx = nextBuff[nextBuffIx];
                 }
             }
@@ -480,9 +485,9 @@ public class OptimizedHybridHashJoin {
 
     public void initProbe() throws HyracksDataException {
 
-        sPartBuffs = new ByteBuffer[numOfSpilledParts];
+        sPartBuffs = new IFrame[numOfSpilledParts];
         for (int i = 0; i < numOfSpilledParts; i++) {
-            sPartBuffs[i] = ctx.allocateFrame();
+            sPartBuffs[i] = new VSizeFrame(ctx);
         }
         curPBuff = new int[numOfPartitions];
         int nextBuffIxToAlloc = 0;
@@ -495,12 +500,12 @@ public class OptimizedHybridHashJoin {
         probePSizeInTups = new int[numOfPartitions];
         probeRFWriters = new RunFileWriter[numOfPartitions];
 
-        probeResBuff = ctx.allocateFrame();
+        probeResBuff = new VSizeFrame(ctx);
 
-        probeTupAppenderToResident = new FrameTupleAppender(ctx.getFrameSize());
+        probeTupAppenderToResident = new FrameTupleAppender();
         probeTupAppenderToResident.reset(probeResBuff, true);
 
-        probeTupAppenderToSpilled = new FrameTupleAppender(ctx.getFrameSize());
+        probeTupAppenderToSpilled = new FrameTupleAppender();
 
     }
 
@@ -517,21 +522,20 @@ public class OptimizedHybridHashJoin {
             inMemJoiner.join(buffer, writer);
             return;
         }
-        ByteBuffer buff = null;
         for (int i = 0; i < tupleCount; ++i) {
             int pid = probeHpc.partition(accessorProbe, i, numOfPartitions);
 
-            if (buildPSizeInTups[pid] > 0) { //Tuple has potential match from previous phase
+            if (buildPSizeInTups[pid] > 0 || isLeftOuter) { //Tuple has potential match from previous phase
                 if (pStatus.get(pid)) { //pid is Spilled
                     boolean needToClear = false;
-                    buff = sPartBuffs[curPBuff[pid]];
+                    IFrame frame = sPartBuffs[curPBuff[pid]];
                     while (true) {
-                        probeTupAppenderToSpilled.reset(buff, needToClear);
+                        probeTupAppenderToSpilled.reset(frame, needToClear);
                         if (probeTupAppenderToSpilled.append(accessorProbe, i)) {
                             break;
                         }
-                        probeWrite(pid, buff);
-                        buff.clear();
+                        probeWrite(pid, frame.getBuffer());
+                        frame.reset();
                         needToClear = true;
                     }
                 } else { //pid is Resident
@@ -539,7 +543,7 @@ public class OptimizedHybridHashJoin {
                         if (probeTupAppenderToResident.append(accessorProbe, i)) {
                             break;
                         }
-                        inMemJoiner.join(probeResBuff, writer);
+                        inMemJoiner.join(probeResBuff.getBuffer(), writer);
                         probeTupAppenderToResident.reset(probeResBuff, true);
                     }
 
@@ -551,13 +555,13 @@ public class OptimizedHybridHashJoin {
 
     }
 
-    public void closeProbe(IFrameWriter writer) throws HyracksDataException { //We do NOT join the spilled partitions here, that decision is made at the descriptor level (which join technique to use)
-        inMemJoiner.join(probeResBuff, writer);
+    public void closeProbe(IFrameWriter writer) throws
+            HyracksDataException { //We do NOT join the spilled partitions here, that decision is made at the descriptor level (which join technique to use)
+        inMemJoiner.join(probeResBuff.getBuffer(), writer);
         inMemJoiner.closeJoin(writer);
 
-        ByteBuffer buff = null;
         for (int pid = pStatus.nextSetBit(0); pid >= 0; pid = pStatus.nextSetBit(pid + 1)) {
-            buff = sPartBuffs[curPBuff[pid]];
+            ByteBuffer buff = sPartBuffs[curPBuff[pid]].getBuffer();
             accessorProbe.reset(buff);
             if (accessorProbe.getTupleCount() > 0) {
                 probeWrite(pid, buff);
@@ -651,10 +655,10 @@ public class OptimizedHybridHashJoin {
 
         double avgBuildSpSz = sumOfBuildSpilledSizes / numOfSpilledPartitions;
         double avgProbeSpSz = sumOfProbeSpilledSizes / numOfSpilledPartitions;
-        String s = "Resident Partitions:\t" + numOfResidentPartitions + "\nSpilled Partitions:\t"
-                + numOfSpilledPartitions + "\nAvg Build Spilled Size:\t" + avgBuildSpSz + "\nAvg Probe Spilled Size:\t"
-                + avgProbeSpSz + "\nIn-Memory Tups:\t" + numOfInMemTups + "\nNum of Free Buffers:\t"
-                + freeFramesCounter;
+        String s =
+                "Resident Partitions:\t" + numOfResidentPartitions + "\nSpilled Partitions:\t" + numOfSpilledPartitions
+                        + "\nAvg Build Spilled Size:\t" + avgBuildSpSz + "\nAvg Probe Spilled Size:\t" + avgProbeSpSz
+                        + "\nIn-Memory Tups:\t" + numOfInMemTups + "\nNum of Free Buffers:\t" + freeFramesCounter;
         return s;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 0494288..840eb75 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -21,6 +21,8 @@ import java.nio.ByteBuffer;
 import java.util.BitSet;
 import java.util.logging.Logger;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
@@ -368,7 +370,7 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
 
             IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
                 private BuildAndPartitionTaskState state;
-                private ByteBuffer rPartbuff = ctx.allocateFrame();
+                private IFrame rPartbuff = new VSizeFrame(ctx);
 
                 @Override
                 public void open() throws HyracksDataException {
@@ -397,13 +399,14 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
 
                     BitSet partitionStatus = state.hybridHJ.getPartitionStatus();
 
-                    rPartbuff.clear();
+                    rPartbuff.reset();
                     for (int pid = partitionStatus.nextSetBit(0); pid >= 0; pid = partitionStatus.nextSetBit(pid + 1)) {
 
                         RunFileReader bReader = state.hybridHJ.getBuildRFReader(pid);
                         RunFileReader pReader = state.hybridHJ.getProbeRFReader(pid);
 
-                        if (bReader == null || pReader == null) { //either of sides (or both) does not have any tuple, thus no need for joining (no potential match)
+                        if (bReader == null || pReader
+                                == null) { //either of sides (or both) does not have any tuple, thus no need for joining (no potential match)
                             continue;
                         }
                         int bSize = state.hybridHJ.getBuildPartitionSizeInTup(pid);
@@ -423,10 +426,14 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
                     ITuplePartitionComputer buildHpc = new FieldHashPartitionComputerFamily(buildKeys,
                             hashFunctionGeneratorFactories).createPartitioner(level);
 
-                    long buildPartSize = wasReversed ? (ohhj.getProbePartitionSize(pid) / ctx.getFrameSize()) : (ohhj
-                            .getBuildPartitionSize(pid) / ctx.getFrameSize());
-                    long probePartSize = wasReversed ? (ohhj.getBuildPartitionSize(pid) / ctx.getFrameSize()) : (ohhj
-                            .getProbePartitionSize(pid) / ctx.getFrameSize());
+                    long buildPartSize = wasReversed ?
+                            (ohhj.getProbePartitionSize(pid) / ctx.getInitialFrameSize()) :
+                            (ohhj
+                                    .getBuildPartitionSize(pid) / ctx.getInitialFrameSize());
+                    long probePartSize = wasReversed ?
+                            (ohhj.getBuildPartitionSize(pid) / ctx.getInitialFrameSize()) :
+                            (ohhj
+                                    .getProbePartitionSize(pid) / ctx.getInitialFrameSize());
 
                     LOGGER.fine("\n>>>Joining Partition Pairs (thread_id " + Thread.currentThread().getId() + ") (pid "
                             + pid + ") - (level " + level + ") - wasReversed " + wasReversed + " - BuildSize:\t"
@@ -437,7 +444,8 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
                     if (!skipInMemoryHJ && (buildPartSize < state.memForJoin)
                             || (probePartSize < state.memForJoin && !isLeftOuter)) {
                         int tabSize = -1;
-                        if (!forceRR && (isLeftOuter || (buildPartSize < probePartSize))) { //Case 1.1 - InMemHJ (wout Role-Reversal)
+                        if (!forceRR && (isLeftOuter || (buildPartSize
+                                < probePartSize))) { //Case 1.1 - InMemHJ (wout Role-Reversal)
                             LOGGER.fine("\t>>>Case 1.1 (IsLeftOuter || buildSize<probe) AND ApplyInMemHJ - [Level "
                                     + level + "]");
                             tabSize = wasReversed ? ohhj.getProbePartitionSizeInTup(pid) : ohhj
@@ -450,8 +458,9 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
                             applyInMemHashJoin(buildKeys, probeKeys, tabSize, probeRd, buildRd, probeHpc, buildHpc,
                                     buildSideReader, probeSideReader, false, pid); //checked-confirmed
                         } else { //Case 1.2 - InMemHJ with Role Reversal
-                            LOGGER.fine("\t>>>Case 1.2. (NoIsLeftOuter || probe<build) AND ApplyInMemHJ WITH RoleReversal - [Level "
-                                    + level + "]");
+                            LOGGER.fine(
+                                    "\t>>>Case 1.2. (NoIsLeftOuter || probe<build) AND ApplyInMemHJ WITH RoleReversal - [Level "
+                                            + level + "]");
                             tabSize = wasReversed ? ohhj.getBuildPartitionSizeInTup(pid) : ohhj
                                     .getProbePartitionSizeInTup(pid);
                             if (tabSize == 0) {
@@ -467,7 +476,8 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
                     else {
                         LOGGER.fine("\t>>>Case 2. ApplyRecursiveHHJ - [Level " + level + "]");
                         OptimizedHybridHashJoin rHHj;
-                        if (!forceRR && (isLeftOuter || buildPartSize < probePartSize)) { //Case 2.1 - Recursive HHJ (wout Role-Reversal)
+                        if (!forceRR && (isLeftOuter
+                                || buildPartSize < probePartSize)) { //Case 2.1 - Recursive HHJ (wout Role-Reversal)
                             LOGGER.fine("\t\t>>>Case 2.1 - RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
                                     + level + "]");
                             int n = getNumberOfPartitions(state.memForJoin, (int) buildPartSize, fudgeFactor,
@@ -478,18 +488,18 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
 
                             buildSideReader.open();
                             rHHj.initBuild();
-                            rPartbuff.clear();
+                            rPartbuff.reset();
                             while (buildSideReader.nextFrame(rPartbuff)) {
-                                rHHj.build(rPartbuff);
+                                rHHj.build(rPartbuff.getBuffer());
                             }
 
                             rHHj.closeBuild();
 
                             probeSideReader.open();
                             rHHj.initProbe();
-                            rPartbuff.clear();
+                            rPartbuff.reset();
                             while (probeSideReader.nextFrame(rPartbuff)) {
-                                rHHj.probe(rPartbuff, writer);
+                                rHHj.probe(rPartbuff.getBuffer(), writer);
                             }
                             rHHj.closeProbe(writer);
 
@@ -499,10 +509,13 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
                                     : maxAfterProbeSize;
 
                             BitSet rPStatus = rHHj.getPartitionStatus();
-                            if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD * beforeMax))) { //Case 2.1.1 - Keep applying HHJ
-                                LOGGER.fine("\t\t>>>Case 2.1.1 - KEEP APPLYING RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
-                                        + level + "]");
-                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+                            if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD
+                                    * beforeMax))) { //Case 2.1.1 - Keep applying HHJ
+                                LOGGER.fine(
+                                        "\t\t>>>Case 2.1.1 - KEEP APPLYING RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+                                                + level + "]");
+                                for (int rPid = rPStatus.nextSetBit(0);
+                                     rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
                                     RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
                                     RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
 
@@ -510,13 +523,16 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
                                         continue;
                                     }
 
-                                    joinPartitionPair(rHHj, rbrfw, rprfw, rPid, afterMax, (level + 1), false); //checked-confirmed
+                                    joinPartitionPair(rHHj, rbrfw, rprfw, rPid, afterMax, (level + 1),
+                                            false); //checked-confirmed
                                 }
 
                             } else { //Case 2.1.2 - Switch to NLJ
-                                LOGGER.fine("\t\t>>>Case 2.1.2 - SWITCHED to NLJ RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
-                                        + level + "]");
-                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+                                LOGGER.fine(
+                                        "\t\t>>>Case 2.1.2 - SWITCHED to NLJ RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+                                                + level + "]");
+                                for (int rPid = rPStatus.nextSetBit(0);
+                                     rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
                                     RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
                                     RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
 
@@ -547,16 +563,16 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
 
                             probeSideReader.open();
                             rHHj.initBuild();
-                            rPartbuff.clear();
+                            rPartbuff.reset();
                             while (probeSideReader.nextFrame(rPartbuff)) {
-                                rHHj.build(rPartbuff);
+                                rHHj.build(rPartbuff.getBuffer());
                             }
                             rHHj.closeBuild();
                             rHHj.initProbe();
                             buildSideReader.open();
-                            rPartbuff.clear();
+                            rPartbuff.reset();
                             while (buildSideReader.nextFrame(rPartbuff)) {
-                                rHHj.probe(rPartbuff, writer);
+                                rHHj.probe(rPartbuff.getBuffer(), writer);
                             }
                             rHHj.closeProbe(writer);
                             int maxAfterBuildSize = rHHj.getMaxBuildPartitionSize();
@@ -565,10 +581,12 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
                                     : maxAfterProbeSize;
                             BitSet rPStatus = rHHj.getPartitionStatus();
 
-                            if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD * beforeMax))) { //Case 2.2.1 - Keep applying HHJ
+                            if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD
+                                    * beforeMax))) { //Case 2.2.1 - Keep applying HHJ
                                 LOGGER.fine("\t\t>>>Case 2.2.1 - KEEP APPLYING RecursiveHHJ WITH RoleReversal - [Level "
                                         + level + "]");
-                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+                                for (int rPid = rPStatus.nextSetBit(0);
+                                     rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
                                     RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
                                     RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
 
@@ -576,12 +594,15 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
                                         continue;
                                     }
 
-                                    joinPartitionPair(rHHj, rprfw, rbrfw, rPid, afterMax, (level + 1), true); //checked-confirmed
+                                    joinPartitionPair(rHHj, rprfw, rbrfw, rPid, afterMax, (level + 1),
+                                            true); //checked-confirmed
                                 }
                             } else { //Case 2.2.2 - Switch to NLJ
-                                LOGGER.fine("\t\t>>>Case 2.2.2 - SWITCHED to NLJ RecursiveHHJ WITH RoleReversal - [Level "
-                                        + level + "]");
-                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+                                LOGGER.fine(
+                                        "\t\t>>>Case 2.2.2 - SWITCHED to NLJ RecursiveHHJ WITH RoleReversal - [Level "
+                                                + level + "]");
+                                for (int rPid = rPStatus.nextSetBit(0);
+                                     rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
                                     RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
                                     RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
 
@@ -611,27 +632,27 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
                         ITuplePartitionComputer hpcRepSmaller, RunFileReader bReader, RunFileReader pReader,
                         boolean reverse, int pid) throws HyracksDataException {
                     ISerializableTable table = new SerializableHashTable(tabSize, ctx);
-                    InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tabSize, new FrameTupleAccessor(
-                            ctx.getFrameSize(), probeRDesc), hpcRepLarger, new FrameTupleAccessor(ctx.getFrameSize(),
-                            buildRDesc), hpcRepSmaller, new FrameTuplePairComparator(pKeys, bKeys, comparators),
-                            isLeftOuter, nullWriters1, table, predEvaluator, reverse);
+                    InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tabSize, new FrameTupleAccessor(probeRDesc),
+                            hpcRepLarger, new FrameTupleAccessor(buildRDesc), hpcRepSmaller,
+                            new FrameTuplePairComparator(pKeys, bKeys, comparators), isLeftOuter, nullWriters1, table,
+                            predEvaluator, reverse);
 
                     bReader.open();
-                    rPartbuff.clear();
+                    rPartbuff.reset();
                     while (bReader.nextFrame(rPartbuff)) {
-                        ByteBuffer copyBuffer = ctx.allocateFrame(); //We need to allocate a copyBuffer, because this buffer gets added to the buffers list in the InMemoryHashJoin
-                        FrameUtils.copy(rPartbuff, copyBuffer);
-                        FrameUtils.makeReadable(copyBuffer);
+                        ByteBuffer copyBuffer = ctx
+                                .allocateFrame(rPartbuff.getFrameSize()); //We need to allocate a copyBuffer, because this buffer gets added to the buffers list in the InMemoryHashJoin
+                        FrameUtils.copyAndFlip(rPartbuff.getBuffer(), copyBuffer);
                         joiner.build(copyBuffer);
-                        rPartbuff.clear();
+                        rPartbuff.reset();
                     }
                     bReader.close();
-                    rPartbuff.clear();
+                    rPartbuff.reset();
                     // probe
                     pReader.open();
                     while (pReader.nextFrame(rPartbuff)) {
-                        joiner.join(rPartbuff, writer);
-                        rPartbuff.clear();
+                        joiner.join(rPartbuff.getBuffer(), writer);
+                        rPartbuff.reset();
                     }
                     pReader.close();
                     joiner.closeJoin(writer);
@@ -640,27 +661,26 @@ public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorD
                 private void applyNestedLoopJoin(RecordDescriptor outerRd, RecordDescriptor innerRd, int memorySize,
                         RunFileReader outerReader, RunFileReader innerReader, ITuplePairComparator nljComparator,
                         boolean reverse) throws HyracksDataException {
-                    NestedLoopJoin nlj = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), outerRd),
-                            new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize,
+                    NestedLoopJoin nlj = new NestedLoopJoin(ctx,
+                            new FrameTupleAccessor(outerRd),
+                            new FrameTupleAccessor(innerRd), nljComparator, memorySize,
                             predEvaluator, isLeftOuter, nullWriters1);
                     nlj.setIsReversed(reverse);
 
-                    ByteBuffer cacheBuff = ctx.allocateFrame();
+                    IFrame cacheBuff = new VSizeFrame(ctx);
                     innerReader.open();
                     while (innerReader.nextFrame(cacheBuff)) {
-                        FrameUtils.makeReadable(cacheBuff);
-                        nlj.cache(cacheBuff);
-                        cacheBuff.clear();
+                        nlj.cache(cacheBuff.getBuffer());
+                        cacheBuff.reset();
                     }
                     nlj.closeCache();
 
-                    ByteBuffer joinBuff = ctx.allocateFrame();
+                    IFrame joinBuff = new VSizeFrame(ctx);
                     outerReader.open();
 
                     while (outerReader.nextFrame(joinBuff)) {
-                        FrameUtils.makeReadable(joinBuff);
-                        nlj.join(joinBuff, writer);
-                        joinBuff.clear();
+                        nlj.join(joinBuff.getBuffer(), writer);
+                        joinBuff.reset();
                     }
 
                     nlj.closeJoin(writer);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
index 12f319f..ffe3abd 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
@@ -15,12 +15,10 @@
 
 package edu.uci.ics.hyracks.dataflow.std.misc;
 
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 
 public class ConstantTupleSourceOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
@@ -41,14 +39,12 @@ public class ConstantTupleSourceOperatorNodePushable extends AbstractUnaryOutput
 
     @Override
     public void initialize() throws HyracksDataException {
-        ByteBuffer writeBuffer = ctx.allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-        appender.reset(writeBuffer, true);
+        FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
         if (fieldSlots != null && tupleData != null && tupleSize > 0)
             appender.append(fieldSlots, tupleData, 0, tupleSize);
         writer.open();
         try {
-            FrameUtils.flushFrame(writeBuffer, writer);
+            appender.flush(writer, false);
         } catch (Exception e) {
             writer.fail();
             throw new HyracksDataException(e);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/LimitOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/LimitOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/LimitOperatorDescriptor.java
index 1fff4fe..4356181 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/LimitOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/LimitOperatorDescriptor.java
@@ -16,6 +16,7 @@ package edu.uci.ics.hyracks.dataflow.std.misc;
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -50,7 +51,7 @@ public class LimitOperatorDescriptor extends AbstractSingleActivityOperatorDescr
 
             @Override
             public void open() throws HyracksDataException {
-                fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptors[0]);
+                fta = new FrameTupleAccessor(recordDescriptors[0]);
                 currentSize = 0;
                 finished = false;
                 writer.open();
@@ -62,16 +63,13 @@ public class LimitOperatorDescriptor extends AbstractSingleActivityOperatorDescr
                     fta.reset(buffer);
                     int count = fta.getTupleCount();
                     if ((currentSize + count) > outputLimit) {
-                        ByteBuffer b = ctx.allocateFrame();
-                        FrameTupleAppender partialAppender = new FrameTupleAppender(ctx.getFrameSize());
-                        partialAppender.reset(b, true);
+                        FrameTupleAppender partialAppender = new FrameTupleAppender(new VSizeFrame(ctx));
                         int copyCount = outputLimit - currentSize;
                         for (int i = 0; i < copyCount; i++) {
-                            partialAppender.append(fta, i);
+                            FrameUtils.appendToWriter(writer, partialAppender, fta, i);
                             currentSize++;
                         }
-                        FrameUtils.makeReadable(b);
-                        FrameUtils.flushFrame(b, writer);
+                        partialAppender.flush(writer,false);
                         finished = true;
                     } else {
                         FrameUtils.flushFrame(buffer, writer);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializerTaskState.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializerTaskState.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializerTaskState.java
index 48de837..9339b34 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializerTaskState.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializerTaskState.java
@@ -19,6 +19,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.TaskId;
@@ -27,6 +28,7 @@ import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
 import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
 
 public class MaterializerTaskState extends AbstractStateObject {
@@ -61,15 +63,13 @@ public class MaterializerTaskState extends AbstractStateObject {
         out.nextFrame(buffer);
     }
 
-    public void writeOut(IFrameWriter writer, ByteBuffer frame) throws HyracksDataException {
+    public void writeOut(IFrameWriter writer, IFrame frame) throws HyracksDataException {
         RunFileReader in = out.createReader();
         writer.open();
         try {
             in.open();
             while (in.nextFrame(frame)) {
-                frame.flip();
-                writer.nextFrame(frame);
-                frame.clear();
+                writer.nextFrame(frame.getBuffer());
             }
             in.close();
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
index 3a405d0..36fdd50 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
@@ -16,6 +16,7 @@ package edu.uci.ics.hyracks.dataflow.std.misc;
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
@@ -25,6 +26,8 @@ import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 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.AbstractUnaryInputSinkOperatorNodePushable;
@@ -107,8 +110,7 @@ public class MaterializingOperatorDescriptor extends AbstractOperatorDescriptor
                 @Override
                 public void close() throws HyracksDataException {
                     state.close();
-                    ByteBuffer frame = ctx.allocateFrame();
-                    state.writeOut(writer, frame);
+                    state.writeOut(writer, new VSizeFrame(ctx));
                 }
 
             };
@@ -166,10 +168,9 @@ public class MaterializingOperatorDescriptor extends AbstractOperatorDescriptor
             return new AbstractUnaryOutputSourceOperatorNodePushable() {
                 @Override
                 public void initialize() throws HyracksDataException {
-                    ByteBuffer frame = ctx.allocateFrame();
                     MaterializerTaskState state = (MaterializerTaskState) ctx.getStateObject(new TaskId(new ActivityId(
                             getOperatorId(), MATERIALIZER_ACTIVITY_ID), partition));
-                    state.writeOut(writer, frame);
+                    state.writeOut(writer, new VSizeFrame(ctx));
                 }
 
                 @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
index b8e1ac8..1f6d965 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
@@ -17,6 +17,7 @@ package edu.uci.ics.hyracks.dataflow.std.misc;
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
 import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
@@ -172,10 +173,9 @@ public class SplitOperatorDescriptor extends AbstractOperatorDescriptor {
 
                 @Override
                 public void initialize() throws HyracksDataException {
-                    ByteBuffer frame = ctx.allocateFrame();
                     MaterializerTaskState state = (MaterializerTaskState) ctx.getStateObject(new TaskId(new ActivityId(
                             getOperatorId(), SPLITTER_MATERIALIZER_ACTIVITY_ID), partition));
-                    state.writeOut(writer, frame);
+                    state.writeOut(writer, new VSizeFrame(ctx));
                 }
 
                 @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
index 42ed59e..3a6b5d2 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
@@ -18,7 +18,9 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -61,17 +63,17 @@ public class ResultWriterOperatorDescriptor extends AbstractSingleActivityOperat
             throws HyracksDataException {
         final IDatasetPartitionManager dpm = ctx.getDatasetPartitionManager();
 
-        final ByteBuffer outputBuffer = ctx.allocateFrame();
+        final IFrame frame = new VSizeFrame(ctx);
 
-        final FrameOutputStream frameOutputStream = new FrameOutputStream(ctx.getFrameSize());
-        frameOutputStream.reset(outputBuffer, true);
+        final FrameOutputStream frameOutputStream = new FrameOutputStream(ctx.getInitialFrameSize());
+        frameOutputStream.reset(frame, true);
         PrintStream printStream = new PrintStream(frameOutputStream);
 
         final RecordDescriptor outRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
         final IResultSerializer resultSerializer = resultSerializerFactory.createResultSerializer(outRecordDesc,
                 printStream);
 
-        final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDesc);
+        final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(outRecordDesc);
 
         return new AbstractUnaryInputSinkOperatorNodePushable() {
             IFrameWriter datasetPartitionWriter;
@@ -94,12 +96,8 @@ public class ResultWriterOperatorDescriptor extends AbstractSingleActivityOperat
                 for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
                     resultSerializer.appendTuple(frameTupleAccessor, tIndex);
                     if (!frameOutputStream.appendTuple()) {
-                        datasetPartitionWriter.nextFrame(outputBuffer);
-                        frameOutputStream.reset(outputBuffer, true);
+                        frameOutputStream.flush(datasetPartitionWriter);
 
-                        /* TODO(madhusudancs): This works under the assumption that no single serialized record is
-                         * longer than the buffer size.
-                         */
                         resultSerializer.appendTuple(frameTupleAccessor, tIndex);
                         frameOutputStream.appendTuple();
                     }
@@ -114,8 +112,7 @@ public class ResultWriterOperatorDescriptor extends AbstractSingleActivityOperat
             @Override
             public void close() throws HyracksDataException {
                 if (frameOutputStream.getTupleCount() > 0) {
-                    datasetPartitionWriter.nextFrame(outputBuffer);
-                    frameOutputStream.reset(outputBuffer, true);
+                    frameOutputStream.flush(datasetPartitionWriter);
                 }
                 datasetPartitionWriter.close();
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractFrameSorter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractFrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractFrameSorter.java
new file mode 100644
index 0000000..2a1a403
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractFrameSorter.java
@@ -0,0 +1,186 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
+
+public abstract class AbstractFrameSorter implements IFrameSorter {
+
+    protected Logger LOGGER = Logger.getLogger(AbstractFrameSorter.class.getName());
+    static final int PTR_SIZE = 4;
+    static final int ID_FRAMEID = 0;
+    static final int ID_TUPLE_START = 1;
+    static final int ID_TUPLE_END = 2;
+    static final int ID_NORMAL_KEY = 3;
+
+    protected final int[] sortFields;
+    protected final IBinaryComparator[] comparators;
+    protected final INormalizedKeyComputer nkc;
+    protected final IFrameBufferManager bufferManager;
+    protected final FrameTupleAccessor inputTupleAccessor;
+    protected final IFrameTupleAppender outputAppender;
+    protected final IFrame outputFrame;
+    protected final int outputLimit;
+
+    protected int[] tPointers;
+    protected int tupleCount;
+
+    public AbstractFrameSorter(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor) throws HyracksDataException {
+        this(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+                Integer.MAX_VALUE);
+    }
+
+    public AbstractFrameSorter(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, int outputLimit)
+            throws HyracksDataException {
+        this.bufferManager = bufferManager;
+        this.sortFields = sortFields;
+        this.nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
+        this.comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        this.inputTupleAccessor = new FrameTupleAccessor(recordDescriptor);
+        this.outputAppender = new FrameTupleAppender();
+        this.outputFrame = new VSizeFrame(ctx);
+        this.outputLimit = outputLimit;
+    }
+
+    @Override
+    public void reset() throws HyracksDataException {
+        this.tupleCount = 0;
+        this.bufferManager.reset();
+    }
+
+    @Override
+    public boolean insertFrame(ByteBuffer inputBuffer) throws HyracksDataException {
+        if (bufferManager.insertFrame(inputBuffer) >= 0) {
+            return true;
+        }
+        if (getFrameCount() == 0) {
+            throw new HyracksDataException(
+                    "The input frame is too big for the sorting buffer, please allocate bigger buffer size");
+        }
+        return false;
+    }
+
+    @Override
+    public void sort() throws HyracksDataException {
+        tupleCount = 0;
+        for (int i = 0; i < bufferManager.getNumFrames(); ++i) {
+            inputTupleAccessor
+                    .reset(bufferManager.getFrame(i), bufferManager.getFrameStartOffset(i),
+                            bufferManager.getFrameSize(i));
+            tupleCount += inputTupleAccessor.getTupleCount();
+        }
+        if (tPointers == null || tPointers.length < tupleCount * PTR_SIZE) {
+            tPointers = new int[tupleCount * PTR_SIZE];
+        }
+        int ptr = 0;
+        int sfIdx = sortFields[0];
+        for (int i = 0; i < bufferManager.getNumFrames(); ++i) {
+            inputTupleAccessor
+                    .reset(bufferManager.getFrame(i), bufferManager.getFrameStartOffset(i),
+                            bufferManager.getFrameSize(i));
+            int tCount = inputTupleAccessor.getTupleCount();
+            byte[] array = inputTupleAccessor.getBuffer().array();
+            for (int j = 0; j < tCount; ++j) {
+                int tStart = inputTupleAccessor.getTupleStartOffset(j);
+                int tEnd = inputTupleAccessor.getTupleEndOffset(j);
+                tPointers[ptr * PTR_SIZE + ID_FRAMEID] = i;
+                tPointers[ptr * PTR_SIZE + ID_TUPLE_START] = tStart;
+                tPointers[ptr * PTR_SIZE + ID_TUPLE_END] = tEnd;
+                int f0StartRel = inputTupleAccessor.getFieldStartOffset(j, sfIdx);
+                int f0EndRel = inputTupleAccessor.getFieldEndOffset(j, sfIdx);
+                int f0Start = f0StartRel + tStart + inputTupleAccessor.getFieldSlotsLength();
+                tPointers[ptr * PTR_SIZE + ID_NORMAL_KEY] =
+                        nkc == null ? 0 : nkc.normalize(array, f0Start, f0EndRel - f0StartRel);
+                ++ptr;
+            }
+        }
+        if (tupleCount > 0) {
+            sortTupleReferences();
+        }
+    }
+
+    abstract void sortTupleReferences() throws HyracksDataException;
+
+    @Override
+    public int getFrameCount() {
+        return bufferManager.getNumFrames();
+    }
+
+    @Override
+    public boolean hasRemaining() {
+        return getFrameCount() > 0;
+    }
+
+    @Override
+    public int flush(IFrameWriter writer) throws HyracksDataException {
+        outputAppender.reset(outputFrame, true);
+        int maxFrameSize = outputFrame.getFrameSize();
+        int limit = Math.min(tupleCount, outputLimit);
+        int io = 0;
+        for (int ptr = 0; ptr < limit; ++ptr) {
+            int i = tPointers[ptr * PTR_SIZE + ID_FRAMEID];
+            int tStart = tPointers[ptr * PTR_SIZE + ID_TUPLE_START];
+            int tEnd = tPointers[ptr * PTR_SIZE + ID_TUPLE_END];
+            ByteBuffer buffer = bufferManager.getFrame(i);
+            inputTupleAccessor.reset(buffer, bufferManager.getFrameStartOffset(i), bufferManager.getFrameSize(i));
+
+            int flushed = FrameUtils.appendToWriter(writer, outputAppender, inputTupleAccessor, tStart, tEnd);
+            if (flushed > 0) {
+                maxFrameSize = Math.max(maxFrameSize, flushed);
+                io++;
+            }
+        }
+        maxFrameSize = Math.max(maxFrameSize, outputFrame.getFrameSize());
+        outputAppender.flush(writer, true);
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine(
+                    "Flushed records:" + limit + " out of " + tupleCount + "; Flushed through " + (io + 1) + " frames");
+        }
+        return maxFrameSize;
+    }
+
+    @Override
+    public void close() {
+        tupleCount = 0;
+        bufferManager.close();
+        tPointers = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java
new file mode 100644
index 0000000..1dd35a8
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+
+public abstract class AbstractSortRunGenerator implements IRunGenerator {
+    protected final List<RunAndMaxFrameSizePair> runAndMaxSizes;
+
+    public AbstractSortRunGenerator() {
+        runAndMaxSizes = new LinkedList<>();
+    }
+
+    abstract public ISorter getSorter() throws HyracksDataException;
+
+    @Override
+    public void open() throws HyracksDataException {
+        runAndMaxSizes.clear();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (getSorter().hasRemaining()) {
+            if (runAndMaxSizes.size() <= 0) {
+                getSorter().sort();
+            } else {
+                flushFramesToRun();
+            }
+        }
+    }
+
+    abstract protected RunFileWriter getRunFileWriter() throws HyracksDataException;
+
+    abstract protected IFrameWriter getFlushableFrameWriter(RunFileWriter writer) throws HyracksDataException;
+
+    protected void flushFramesToRun() throws HyracksDataException {
+        getSorter().sort();
+        RunFileWriter runWriter = getRunFileWriter();
+        IFrameWriter flushWriter = getFlushableFrameWriter(runWriter);
+        flushWriter.open();
+        int maxFlushedFrameSize;
+        try {
+            maxFlushedFrameSize = getSorter().flush(flushWriter);
+        } finally {
+            flushWriter.close();
+        }
+        runAndMaxSizes.add(new RunAndMaxFrameSizePair(runWriter.createReader(), maxFlushedFrameSize));
+        getSorter().reset();
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+    }
+
+    @Override
+    public List<RunAndMaxFrameSizePair> getRuns() {
+        return runAndMaxSizes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
new file mode 100644
index 0000000..0c1c622
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
@@ -0,0 +1,197 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.api.job.JobId;
+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.AbstractStateObject;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+
+public abstract class AbstractSorterOperatorDescriptor extends AbstractOperatorDescriptor {
+
+    private static final Logger LOGGER = Logger.getLogger(AbstractSorterOperatorDescriptor.class.getName());
+
+    private static final long serialVersionUID = 1L;
+
+    protected static final int SORT_ACTIVITY_ID = 0;
+    protected static final int MERGE_ACTIVITY_ID = 1;
+
+    protected final int[] sortFields;
+    protected final INormalizedKeyComputerFactory firstKeyNormalizerFactory;
+    protected final IBinaryComparatorFactory[] comparatorFactories;
+    protected final int framesLimit;
+
+    public AbstractSorterOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.framesLimit = framesLimit;
+        this.sortFields = sortFields;
+        this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
+        this.comparatorFactories = comparatorFactories;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    public abstract SortActivity getSortActivity(ActivityId id);
+
+    public abstract MergeActivity getMergeActivity(ActivityId id);
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        SortActivity sa = getSortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
+        MergeActivity ma = getMergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
+
+        builder.addActivity(this, sa);
+        builder.addSourceEdge(0, sa, 0);
+
+        builder.addActivity(this, ma);
+        builder.addTargetEdge(0, ma, 0);
+
+        builder.addBlockingEdge(sa, ma);
+    }
+
+    public static class SortTaskState extends AbstractStateObject {
+        public List<RunAndMaxFrameSizePair> runAndMaxFrameSizePairs;
+        public ISorter sorter;
+
+        public SortTaskState(JobId jobId, TaskId taskId) {
+            super(jobId, taskId);
+        }
+
+        @Override
+        public void toBytes(DataOutput out) throws IOException {
+
+        }
+
+        @Override
+        public void fromBytes(DataInput in) throws IOException {
+
+        }
+    }
+
+    protected abstract class SortActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public SortActivity(ActivityId id) {
+            super(id);
+        }
+
+        protected abstract AbstractSortRunGenerator getRunGenerator(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider) throws HyracksDataException;
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
+                private AbstractSortRunGenerator runGen;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    runGen = getRunGenerator(ctx, recordDescProvider);
+                    runGen.open();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    runGen.nextFrame(buffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    SortTaskState state = new SortTaskState(ctx.getJobletContext().getJobId(),
+                            new TaskId(getActivityId(), partition));
+                    runGen.close();
+                    state.runAndMaxFrameSizePairs = runGen.getRuns();
+                    state.sorter = runGen.getSorter();
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("InitialNumberOfRuns:" + runGen.getRuns().size());
+                    }
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    runGen.fail();
+                }
+            };
+            return op;
+        }
+    }
+
+    protected abstract class MergeActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public MergeActivity(ActivityId id) {
+            super(id);
+        }
+
+        protected abstract ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter,
+                List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+                int necessaryFrames);
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
+
+                @Override
+                public void initialize() throws HyracksDataException {
+                    SortTaskState state = (SortTaskState) ctx
+                            .getStateObject(new TaskId(new ActivityId(getOperatorId(), SORT_ACTIVITY_ID), partition));
+                    List<RunAndMaxFrameSizePair> runs = state.runAndMaxFrameSizePairs;
+                    ISorter sorter = state.sorter;
+                    IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+                    for (int i = 0; i < comparatorFactories.length; ++i) {
+                        comparators[i] = comparatorFactories[i].createBinaryComparator();
+                    }
+                    INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ?
+                            null :
+                            firstKeyNormalizerFactory.createNormalizedKeyComputer();
+                    ExternalSortRunMerger merger = getSortRunMerger(ctx, recordDescProvider, writer, sorter, runs,
+                            comparators, nmkComputer, framesLimit);
+                    merger.process();
+                }
+            };
+            return op;
+        }
+    }
+
+}


[08/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
deleted file mode 100644
index 110bddb..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
+++ /dev/null
@@ -1,717 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-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;
-
-/**
- * @author pouria Implements Memory Manager based on creating Binary Search Tree
- *         (BST) while Free slot size is the key for the BST nodes. Each node in
- *         BST shows a class of free slots, while all the free slots within a
- *         class have same lengths. Slots in a class are stored as a LinkedList,
- *         whose head is the BST node, corresponding to that class. BST is not
- *         stored as a separate data structure, but the free slots in the memory
- *         are used to hold BST nodes. Each BST node has the logical structure,
- *         defined in the BSTNodeUtil class.
- */
-public class BSTMemMgr implements IMemoryManager {
-
-    private final IHyracksTaskContext ctx;
-    public static int frameSize;
-
-    private ByteBuffer[] frames;
-    private ByteBuffer convertBuffer;
-    private Slot root;
-    private Slot result; // A reusable object to hold one node returned as
-                         // method result
-    private Slot insertSlot; // A reusable object to hold one node within insert
-                             // process
-    private Slot lastLeftParent; // A reusable object for the search process
-    private Slot lastLeft; // A reusable object for the search process
-    private Slot parent; // A reusable object for the search process
-
-    private Slot[] parentRes;
-    private int lastFrame;
-
-    public BSTMemMgr(IHyracksTaskContext ctx, int memSize) {
-        this.ctx = ctx;
-        frameSize = ctx.getFrameSize();
-        convertBuffer = ByteBuffer.allocate(4);
-        frames = new ByteBuffer[memSize];
-        lastFrame = -1;
-        root = new Slot();
-        insertSlot = new Slot();
-        result = new Slot();
-        lastLeftParent = new Slot();
-        lastLeft = new Slot();
-        parent = new Slot();
-        parentRes = new Slot[] { new Slot(), new Slot() };
-    }
-
-    /**
-     * result is the container sent by the caller to hold the results
-     */
-    @Override
-    public void allocate(int length, Slot result) throws HyracksDataException {
-        search(length, parentRes);
-        if (parentRes[1].isNull()) {
-            addFrame(parentRes);
-            if (parentRes[1].isNull()) {
-                return;
-            }
-        }
-
-        int sl = BSTNodeUtil.getLength(parentRes[1], frames, convertBuffer);
-        int acLen = BSTNodeUtil.getActualLength(length);
-        if (shouldSplit(sl, acLen)) {
-            int[] s = split(parentRes[1], parentRes[0], acLen);
-            int insertLen = BSTNodeUtil.getLength(s[2], s[3], frames, convertBuffer);
-            insert(s[2], s[3], insertLen); // inserting second half of the split
-                                           // slot
-            BSTNodeUtil.setHeaderFooter(s[0], s[1], length, false, frames);
-            result.set(s[0], s[1]);
-            return;
-        }
-        allocate(parentRes[1], parentRes[0], length, result);
-    }
-
-    @Override
-    public int unallocate(Slot s) throws HyracksDataException {
-        int usedLen = BSTNodeUtil.getLength(s, frames, convertBuffer);
-        int actualLen = BSTNodeUtil.getActualLength(usedLen);
-        int fix = s.getFrameIx();
-        int off = s.getOffset();
-
-        int prevMemSlotFooterOffset = ((off - BSTNodeUtil.HEADER_SIZE) >= 0 ? (off - BSTNodeUtil.HEADER_SIZE)
-                : BSTNodeUtil.INVALID_INDEX);
-        int t = off + 2 * BSTNodeUtil.HEADER_SIZE + actualLen;
-        int nextMemSlotHeaderOffset = (t < frameSize ? t : BSTNodeUtil.INVALID_INDEX);
-        // Remember: next and prev memory slots have the same frame index as the
-        // unallocated slot
-        if (!isNodeNull(fix, prevMemSlotFooterOffset) && BSTNodeUtil.isFree(fix, prevMemSlotFooterOffset, frames)) {
-            int leftLength = BSTNodeUtil.getLength(fix, prevMemSlotFooterOffset, frames, convertBuffer);
-            removeFromList(fix, prevMemSlotFooterOffset - leftLength - BSTNodeUtil.HEADER_SIZE);
-            int concatLength = actualLen + leftLength + 2 * BSTNodeUtil.HEADER_SIZE;
-            if (!isNodeNull(fix, nextMemSlotHeaderOffset) && BSTNodeUtil.isFree(fix, nextMemSlotHeaderOffset, frames)) {
-                removeFromList(fix, nextMemSlotHeaderOffset);
-                concatLength += BSTNodeUtil.getLength(fix, nextMemSlotHeaderOffset, frames, convertBuffer) + 2
-                        * BSTNodeUtil.HEADER_SIZE;
-            }
-            insert(fix, prevMemSlotFooterOffset - leftLength - BSTNodeUtil.HEADER_SIZE, concatLength); // newly
-                                                                                                       // (merged)
-                                                                                                       // slot
-                                                                                                       // starts
-                                                                                                       // at
-                                                                                                       // the
-                                                                                                       // prev
-                                                                                                       // slot
-                                                                                                       // offset
-            return concatLength;
-
-        } else if (!isNodeNull(fix, nextMemSlotHeaderOffset)
-                && BSTNodeUtil.isFree(fix, nextMemSlotHeaderOffset, frames)) {
-            removeFromList(fix, nextMemSlotHeaderOffset);
-            int concatLength = actualLen + BSTNodeUtil.getLength(fix, nextMemSlotHeaderOffset, frames, convertBuffer)
-                    + 2 * BSTNodeUtil.HEADER_SIZE;
-            insert(fix, off, concatLength); // newly (merged) slot starts at the
-                                            // unallocated slot offset
-            return concatLength;
-        }
-        // unallocated slot is not merging with any neighbor
-        insert(fix, off, actualLen);
-        return actualLen;
-    }
-
-    @Override
-    public boolean readTuple(int frameIx, int offset, FrameTupleAppender dest) {
-        int offToRead = offset + BSTNodeUtil.HEADER_SIZE;
-        int length = BSTNodeUtil.getLength(frameIx, offset, frames, convertBuffer);
-        return dest.append(frames[frameIx].array(), offToRead, length);
-    }
-
-    @Override
-    public boolean writeTuple(int frameIx, int offset, FrameTupleAccessor src, int tIndex) {
-        int offToCopy = offset + BSTNodeUtil.HEADER_SIZE;
-        int tStartOffset = src.getTupleStartOffset(tIndex);
-        int tEndOffset = src.getTupleEndOffset(tIndex);
-        int tupleLength = tEndOffset - tStartOffset;
-        ByteBuffer srcBuffer = src.getBuffer();
-        System.arraycopy(srcBuffer.array(), tStartOffset, frames[frameIx].array(), offToCopy, tupleLength);
-        return true;
-    }
-
-    @Override
-    public ByteBuffer getFrame(int frameIndex) {
-        return frames[frameIndex];
-    }
-
-    @Override
-    public void close() {
-        //clean up all frames
-        for (int i = 0; i < frames.length; i++)
-            frames[i] = null;
-    }
-
-    /**
-     * @param parentResult
-     *            is the container passed by the caller to contain the results
-     * @throws HyracksDataException
-     */
-    private void addFrame(Slot[] parentResult) throws HyracksDataException {
-        clear(parentResult);
-        if ((lastFrame + 1) >= frames.length) {
-            return;
-        }
-        frames[++lastFrame] = allocateFrame();
-        int l = frameSize - 2 * BSTNodeUtil.HEADER_SIZE;
-        BSTNodeUtil.setHeaderFooter(lastFrame, 0, l, true, frames);
-        initNewNode(lastFrame, 0);
-
-        parentResult[1].copy(root);
-        if (parentResult[1].isNull()) { // root is null
-            root.set(lastFrame, 0);
-            initNewNode(root.getFrameIx(), root.getOffset());
-            parentResult[1].copy(root);
-            return;
-        }
-
-        while (!parentResult[1].isNull()) {
-            if (BSTNodeUtil.getLength(parentResult[1], frames, convertBuffer) == l) {
-                append(parentResult[1].getFrameIx(), parentResult[1].getOffset(), lastFrame, 0);
-                parentResult[1].set(lastFrame, 0);
-                return;
-            }
-            if (l < BSTNodeUtil.getLength(parentResult[1], frames, convertBuffer)) {
-                if (isNodeNull(BSTNodeUtil.getLeftChildFrameIx(parentResult[1], frames, convertBuffer),
-                        BSTNodeUtil.getLeftChildOffset(parentResult[1], frames, convertBuffer))) {
-                    BSTNodeUtil.setLeftChild(parentResult[1].getFrameIx(), parentResult[1].getOffset(), lastFrame, 0,
-                            frames);
-                    parentResult[0].copy(parentResult[1]);
-                    parentResult[1].set(lastFrame, 0);
-                    return;
-                } else {
-                    parentResult[0].copy(parentResult[1]);
-                    parentResult[1].set(BSTNodeUtil.getLeftChildFrameIx(parentResult[1], frames, convertBuffer),
-                            BSTNodeUtil.getLeftChildOffset(parentResult[1], frames, convertBuffer));
-                }
-            } else {
-                if (isNodeNull(BSTNodeUtil.getRightChildFrameIx(parentResult[1], frames, convertBuffer),
-                        BSTNodeUtil.getRightChildOffset(parentResult[1], frames, convertBuffer))) {
-                    BSTNodeUtil.setRightChild(parentResult[1].getFrameIx(), parentResult[1].getOffset(), lastFrame, 0,
-                            frames);
-                    parentResult[0].copy(parentResult[1]);
-                    parentResult[1].set(lastFrame, 0);
-                    return;
-                } else {
-                    parentResult[0].copy(parentResult[1]);
-                    parentResult[1].set(BSTNodeUtil.getRightChildFrameIx(parentResult[1], frames, convertBuffer),
-                            BSTNodeUtil.getRightChildOffset(parentResult[1], frames, convertBuffer));
-                }
-            }
-        }
-        throw new HyracksDataException("New Frame could not be added to BSTMemMgr");
-    }
-
-    private void insert(int fix, int off, int length) throws HyracksDataException {
-        BSTNodeUtil.setHeaderFooter(fix, off, length, true, frames);
-        initNewNode(fix, off);
-
-        if (root.isNull()) {
-            root.set(fix, off);
-            return;
-        }
-
-        insertSlot.clear();
-        insertSlot.copy(root);
-        while (!insertSlot.isNull()) {
-            int curSlotLen = BSTNodeUtil.getLength(insertSlot, frames, convertBuffer);
-            if (curSlotLen == length) {
-                append(insertSlot.getFrameIx(), insertSlot.getOffset(), fix, off);
-                return;
-            }
-            if (length < curSlotLen) {
-                int leftChildFIx = BSTNodeUtil.getLeftChildFrameIx(insertSlot, frames, convertBuffer);
-                int leftChildOffset = BSTNodeUtil.getLeftChildOffset(insertSlot, frames, convertBuffer);
-                if (isNodeNull(leftChildFIx, leftChildOffset)) {
-                    initNewNode(fix, off);
-                    BSTNodeUtil.setLeftChild(insertSlot.getFrameIx(), insertSlot.getOffset(), fix, off, frames);
-                    return;
-                } else {
-                    insertSlot.set(leftChildFIx, leftChildOffset);
-                }
-            } else {
-                int rightChildFIx = BSTNodeUtil.getRightChildFrameIx(insertSlot, frames, convertBuffer);
-                int rightChildOffset = BSTNodeUtil.getRightChildOffset(insertSlot, frames, convertBuffer);
-                if (isNodeNull(rightChildFIx, rightChildOffset)) {
-                    initNewNode(fix, off);
-                    BSTNodeUtil.setRightChild(insertSlot.getFrameIx(), insertSlot.getOffset(), fix, off, frames);
-                    return;
-                } else {
-                    insertSlot.set(rightChildFIx, rightChildOffset);
-                }
-            }
-        }
-        throw new HyracksDataException("Failure in node insertion into BST in BSTMemMgr");
-    }
-
-    /**
-     * @param length
-     * @param target
-     *            is the container sent by the caller to hold the results
-     */
-    private void search(int length, Slot[] target) {
-        clear(target);
-        result.clear();
-
-        if (root.isNull()) {
-            return;
-        }
-
-        lastLeftParent.clear();
-        lastLeft.clear();
-        parent.clear();
-        result.copy(root);
-
-        while (!result.isNull()) {
-            if (BSTNodeUtil.getLength(result, frames, convertBuffer) == length) {
-                target[0].copy(parent);
-                target[1].copy(result);
-                return;
-            }
-            if (length < BSTNodeUtil.getLength(result, frames, convertBuffer)) {
-                lastLeftParent.copy(parent);
-                lastLeft.copy(result);
-                parent.copy(result);
-                int fix = BSTNodeUtil.getLeftChildFrameIx(result, frames, convertBuffer);
-                int off = BSTNodeUtil.getLeftChildOffset(result, frames, convertBuffer);
-                result.set(fix, off);
-            } else {
-                parent.copy(result);
-                int fix = BSTNodeUtil.getRightChildFrameIx(result, frames, convertBuffer);
-                int off = BSTNodeUtil.getRightChildOffset(result, frames, convertBuffer);
-                result.set(fix, off);
-            }
-        }
-
-        target[0].copy(lastLeftParent);
-        target[1].copy(lastLeft);
-
-    }
-
-    private void append(int headFix, int headOff, int nodeFix, int nodeOff) {
-        initNewNode(nodeFix, nodeOff);
-
-        int fix = BSTNodeUtil.getNextFrameIx(headFix, headOff, frames, convertBuffer); // frameIx
-        // for
-        // the
-        // current
-        // next
-        // of
-        // head
-        int off = BSTNodeUtil.getNextOffset(headFix, headOff, frames, convertBuffer); // offset
-                                                                                      // for
-                                                                                      // the
-                                                                                      // current
-                                                                                      // next
-                                                                                      // of
-                                                                                      // head
-        BSTNodeUtil.setNext(nodeFix, nodeOff, fix, off, frames);
-
-        if (!isNodeNull(fix, off)) {
-            BSTNodeUtil.setPrev(fix, off, nodeFix, nodeOff, frames);
-        }
-        BSTNodeUtil.setPrev(nodeFix, nodeOff, headFix, headOff, frames);
-        BSTNodeUtil.setNext(headFix, headOff, nodeFix, nodeOff, frames);
-    }
-
-    private int[] split(Slot listHead, Slot parent, int length) {
-        int l2 = BSTNodeUtil.getLength(listHead, frames, convertBuffer) - length - 2 * BSTNodeUtil.HEADER_SIZE;
-        // We split the node after slots-list head
-        if (!isNodeNull(BSTNodeUtil.getNextFrameIx(listHead, frames, convertBuffer),
-                BSTNodeUtil.getNextOffset(listHead, frames, convertBuffer))) {
-            int afterHeadFix = BSTNodeUtil.getNextFrameIx(listHead, frames, convertBuffer);
-            int afterHeadOff = BSTNodeUtil.getNextOffset(listHead, frames, convertBuffer);
-            int afHNextFix = BSTNodeUtil.getNextFrameIx(afterHeadFix, afterHeadOff, frames, convertBuffer);
-            int afHNextOff = BSTNodeUtil.getNextOffset(afterHeadFix, afterHeadOff, frames, convertBuffer);
-            BSTNodeUtil.setNext(listHead.getFrameIx(), listHead.getOffset(), afHNextFix, afHNextOff, frames);
-            if (!isNodeNull(afHNextFix, afHNextOff)) {
-                BSTNodeUtil.setPrev(afHNextFix, afHNextOff, listHead.getFrameIx(), listHead.getOffset(), frames);
-            }
-            int secondOffset = afterHeadOff + length + 2 * BSTNodeUtil.HEADER_SIZE;
-            BSTNodeUtil.setHeaderFooter(afterHeadFix, afterHeadOff, length, true, frames);
-            BSTNodeUtil.setHeaderFooter(afterHeadFix, secondOffset, l2, true, frames);
-
-            return new int[] { afterHeadFix, afterHeadOff, afterHeadFix, secondOffset };
-        }
-        // We split the head
-        int secondOffset = listHead.getOffset() + length + 2 * BSTNodeUtil.HEADER_SIZE;
-        BSTNodeUtil.setHeaderFooter(listHead.getFrameIx(), listHead.getOffset(), length, true, frames);
-        BSTNodeUtil.setHeaderFooter(listHead.getFrameIx(), secondOffset, l2, true, frames);
-
-        fixTreePtrs(listHead.getFrameIx(), listHead.getOffset(), parent.getFrameIx(), parent.getOffset());
-        return new int[] { listHead.getFrameIx(), listHead.getOffset(), listHead.getFrameIx(), secondOffset };
-    }
-
-    private void fixTreePtrs(int nodeFrameIx, int nodeOffset, int parentFrameIx, int parentOffset) {
-        int nodeLeftChildFrameIx = BSTNodeUtil.getLeftChildFrameIx(nodeFrameIx, nodeOffset, frames, convertBuffer);
-        int nodeLeftChildOffset = BSTNodeUtil.getLeftChildOffset(nodeFrameIx, nodeOffset, frames, convertBuffer);
-        int nodeRightChildFrameIx = BSTNodeUtil.getRightChildFrameIx(nodeFrameIx, nodeOffset, frames, convertBuffer);
-        int nodeRightChildOffset = BSTNodeUtil.getRightChildOffset(nodeFrameIx, nodeOffset, frames, convertBuffer);
-
-        int status = -1; // (status==0 if node is left child of parent)
-                         // (status==1 if node is right child of parent)
-        if (!isNodeNull(parentFrameIx, parentOffset)) {
-            int nlen = BSTNodeUtil.getActualLength(BSTNodeUtil
-                    .getLength(nodeFrameIx, nodeOffset, frames, convertBuffer));
-            int plen = BSTNodeUtil.getActualLength(BSTNodeUtil.getLength(parentFrameIx, parentOffset, frames,
-                    convertBuffer));
-            status = ((nlen < plen) ? 0 : 1);
-        }
-
-        if (!isNodeNull(nodeLeftChildFrameIx, nodeLeftChildOffset)
-                && !isNodeNull(nodeRightChildFrameIx, nodeRightChildOffset)) { // Node
-            // has
-            // two
-            // children
-            int pMinFIx = nodeFrameIx;
-            int pMinOff = nodeOffset;
-            int minFIx = nodeRightChildFrameIx;
-            int minOff = nodeRightChildOffset;
-            int nextLeftFIx = BSTNodeUtil.getLeftChildFrameIx(minFIx, minOff, frames, convertBuffer);
-            int nextLeftOff = BSTNodeUtil.getLeftChildOffset(minFIx, minOff, frames, convertBuffer);
-
-            while (!isNodeNull(nextLeftFIx, nextLeftOff)) {
-                pMinFIx = minFIx;
-                pMinOff = minOff;
-                minFIx = nextLeftFIx;
-                minOff = nextLeftOff;
-                nextLeftFIx = BSTNodeUtil.getLeftChildFrameIx(minFIx, minOff, frames, convertBuffer); // min
-                                                                                                      // is
-                                                                                                      // now
-                                                                                                      // pointing
-                                                                                                      // to
-                                                                                                      // current
-                                                                                                      // (old)
-                                                                                                      // next
-                                                                                                      // left
-                nextLeftOff = BSTNodeUtil.getLeftChildOffset(minFIx, minOff, frames, convertBuffer); // min
-                                                                                                     // is
-                                                                                                     // now
-                                                                                                     // pointing
-                                                                                                     // to
-                                                                                                     // current
-                                                                                                     // (old)
-                                                                                                     // next
-                                                                                                     // left
-            }
-
-            if ((nodeRightChildFrameIx == minFIx) && (nodeRightChildOffset == minOff)) { // nrc
-                                                                                         // is
-                                                                                         // the
-                // same as min
-                BSTNodeUtil.setLeftChild(nodeRightChildFrameIx, nodeRightChildOffset, nodeLeftChildFrameIx,
-                        nodeLeftChildOffset, frames);
-            } else { // min is different from nrc
-                int minRightFIx = BSTNodeUtil.getRightChildFrameIx(minFIx, minOff, frames, convertBuffer);
-                int minRightOffset = BSTNodeUtil.getRightChildOffset(minFIx, minOff, frames, convertBuffer);
-                BSTNodeUtil.setRightChild(minFIx, minOff, nodeRightChildFrameIx, nodeRightChildOffset, frames);
-                BSTNodeUtil.setLeftChild(minFIx, minOff, nodeLeftChildFrameIx, nodeLeftChildOffset, frames);
-                BSTNodeUtil.setLeftChild(pMinFIx, pMinOff, minRightFIx, minRightOffset, frames);
-            }
-
-            // Now dealing with the parent
-            if (!isNodeNull(parentFrameIx, parentOffset)) {
-                if (status == 0) {
-                    BSTNodeUtil.setLeftChild(parentFrameIx, parentOffset, minFIx, minOff, frames);
-                } else if (status == 1) {
-                    BSTNodeUtil.setRightChild(parentFrameIx, parentOffset, minFIx, minOff, frames);
-                }
-            } else { // No parent (node was the root)
-                root.set(minFIx, minOff);
-            }
-            return;
-        }
-
-        else if (!isNodeNull(nodeLeftChildFrameIx, nodeLeftChildOffset)) { // Node
-                                                                           // has
-                                                                           // only
-                                                                           // left
-                                                                           // child
-            if (status == 0) {
-                BSTNodeUtil
-                        .setLeftChild(parentFrameIx, parentOffset, nodeLeftChildFrameIx, nodeLeftChildOffset, frames);
-            } else if (status == 1) {
-                BSTNodeUtil.setRightChild(parentFrameIx, parentOffset, nodeLeftChildFrameIx, nodeLeftChildOffset,
-                        frames);
-            } else if (status == -1) { // No parent, so node is root
-                root.set(nodeLeftChildFrameIx, nodeLeftChildOffset);
-            }
-            return;
-        }
-
-        else if (!isNodeNull(nodeRightChildFrameIx, nodeRightChildOffset)) { // Node
-                                                                             // has
-                                                                             // only
-                                                                             // right
-                                                                             // child
-            if (status == 0) {
-                BSTNodeUtil.setLeftChild(parentFrameIx, parentOffset, nodeRightChildFrameIx, nodeRightChildOffset,
-                        frames);
-            } else if (status == 1) {
-                BSTNodeUtil.setRightChild(parentFrameIx, parentOffset, nodeRightChildFrameIx, nodeRightChildOffset,
-                        frames);
-            } else if (status == -1) { // No parent, so node is root
-                root.set(nodeRightChildFrameIx, nodeRightChildOffset);
-            }
-            return;
-        }
-
-        else { // Node is leaf (no children)
-            if (status == 0) {
-                BSTNodeUtil.setLeftChild(parentFrameIx, parentOffset, BSTNodeUtil.INVALID_INDEX,
-                        BSTNodeUtil.INVALID_INDEX, frames);
-            } else if (status == 1) {
-                BSTNodeUtil.setRightChild(parentFrameIx, parentOffset, BSTNodeUtil.INVALID_INDEX,
-                        BSTNodeUtil.INVALID_INDEX, frames);
-            } else { // node was the only node in the tree
-                root.clear();
-            }
-            return;
-        }
-    }
-
-    /**
-     * Allocation with no splitting but padding
-     * 
-     * @param node
-     * @param parent
-     * @param result
-     *            is the container sent by the caller to hold the results
-     */
-    private void allocate(Slot node, Slot parent, int length, Slot result) {
-        int nextFix = BSTNodeUtil.getNextFrameIx(node, frames, convertBuffer);
-        int nextOff = BSTNodeUtil.getNextOffset(node, frames, convertBuffer);
-        if (!isNodeNull(nextFix, nextOff)) {
-            int nextOfNextFIx = BSTNodeUtil.getNextFrameIx(nextFix, nextOff, frames, convertBuffer);
-            int nextOfNextOffset = BSTNodeUtil.getNextOffset(nextFix, nextOff, frames, convertBuffer);
-            BSTNodeUtil.setNext(node.getFrameIx(), node.getOffset(), nextOfNextFIx, nextOfNextOffset, frames);
-            if (!isNodeNull(nextOfNextFIx, nextOfNextOffset)) {
-                BSTNodeUtil.setPrev(nextOfNextFIx, nextOfNextOffset, node.getFrameIx(), node.getOffset(), frames);
-            }
-            BSTNodeUtil.setHeaderFooter(nextFix, nextOff, length, false, frames);
-            result.set(nextFix, nextOff);
-            return;
-        }
-
-        fixTreePtrs(node.getFrameIx(), node.getOffset(), parent.getFrameIx(), parent.getOffset());
-        BSTNodeUtil.setHeaderFooter(node.getFrameIx(), node.getOffset(), length, false, frames);
-        result.copy(node);
-    }
-
-    private void removeFromList(int fix, int off) {
-        int nextFIx = BSTNodeUtil.getNextFrameIx(fix, off, frames, convertBuffer);
-        int nextOffset = BSTNodeUtil.getNextOffset(fix, off, frames, convertBuffer);
-        int prevFIx = BSTNodeUtil.getPrevFrameIx(fix, off, frames, convertBuffer);
-        int prevOffset = BSTNodeUtil.getPrevOffset(fix, off, frames, convertBuffer);
-        if (!isNodeNull(prevFIx, prevOffset) && !isNodeNull(nextFIx, nextOffset)) {
-            BSTNodeUtil.setNext(prevFIx, prevOffset, nextFIx, nextOffset, frames);
-            BSTNodeUtil.setPrev(nextFIx, nextOffset, prevFIx, prevOffset, frames);
-            BSTNodeUtil.setNext(fix, off, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
-            BSTNodeUtil.setPrev(fix, off, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
-            return;
-        }
-        if (!isNodeNull(prevFIx, prevOffset)) {
-            BSTNodeUtil.setNext(prevFIx, prevOffset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
-            BSTNodeUtil.setPrev(fix, off, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
-            return;
-        }
-
-        // We need to find the parent, so we can fix the tree
-        int parentFIx = BSTNodeUtil.INVALID_INDEX;
-        int parentOffset = BSTNodeUtil.INVALID_INDEX;
-        int length = BSTNodeUtil.getActualLength(BSTNodeUtil.getLength(fix, off, frames, convertBuffer));
-        fix = root.getFrameIx();
-        off = root.getOffset();
-        int curLen = BSTNodeUtil.getLength(fix, off, frames, convertBuffer);
-        while (length != curLen) {
-            parentFIx = fix;
-            parentOffset = off;
-            if (length < curLen) {
-                fix = BSTNodeUtil.getLeftChildFrameIx(parentFIx, parentOffset, frames, convertBuffer); // parentFIx
-                // is
-                // now
-                // the
-                // old(current)
-                // fix
-                off = BSTNodeUtil.getLeftChildOffset(parentFIx, parentOffset, frames, convertBuffer); // parentOffset
-                // is
-                // now
-                // the
-                // old(current)
-                // off
-            } else {
-                fix = BSTNodeUtil.getRightChildFrameIx(parentFIx, parentOffset, frames, convertBuffer); // parentFIx
-                // is
-                // now
-                // the
-                // old(current)
-                // fix
-                off = BSTNodeUtil.getRightChildOffset(parentFIx, parentOffset, frames, convertBuffer); // parentOffset
-                // is
-                // now
-                // the
-                // old(current)
-                // off
-            }
-            curLen = BSTNodeUtil.getLength(fix, off, frames, convertBuffer);
-        }
-
-        if (!isNodeNull(nextFIx, nextOffset)) { // it is head of the list (in
-                                                // the
-            // tree)
-            BSTNodeUtil.setPrev(nextFIx, nextOffset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
-            int nodeLeftChildFIx = BSTNodeUtil.getLeftChildFrameIx(fix, off, frames, convertBuffer);
-            int nodeLeftChildOffset = BSTNodeUtil.getLeftChildOffset(fix, off, frames, convertBuffer);
-            int nodeRightChildFix = BSTNodeUtil.getRightChildFrameIx(fix, off, frames, convertBuffer);
-            int nodeRightChildOffset = BSTNodeUtil.getRightChildOffset(fix, off, frames, convertBuffer);
-            BSTNodeUtil.setLeftChild(nextFIx, nextOffset, nodeLeftChildFIx, nodeLeftChildOffset, frames);
-            BSTNodeUtil.setRightChild(nextFIx, nextOffset, nodeRightChildFix, nodeRightChildOffset, frames);
-            if (!isNodeNull(parentFIx, parentOffset)) {
-                int parentLength = BSTNodeUtil.getLength(parentFIx, parentOffset, frames, convertBuffer);
-                if (length < parentLength) {
-                    BSTNodeUtil.setLeftChild(parentFIx, parentOffset, nextFIx, nextOffset, frames);
-                } else {
-                    BSTNodeUtil.setRightChild(parentFIx, parentOffset, nextFIx, nextOffset, frames);
-                }
-            }
-
-            if ((root.getFrameIx() == fix) && (root.getOffset() == off)) {
-                root.set(nextFIx, nextOffset);
-            }
-
-            return;
-        }
-
-        fixTreePtrs(fix, off, parentFIx, parentOffset);
-    }
-
-    private void clear(Slot[] s) {
-        s[0].clear();
-        s[1].clear();
-    }
-
-    private boolean isNodeNull(int frameIx, int offset) {
-        return ((frameIx == BSTNodeUtil.INVALID_INDEX) || (offset == BSTNodeUtil.INVALID_INDEX) || (frames[frameIx] == null));
-    }
-
-    private boolean shouldSplit(int slotLength, int reqLength) {
-        return ((slotLength - reqLength) >= BSTNodeUtil.MINIMUM_FREE_SLOT_SIZE);
-    }
-
-    private void initNewNode(int frameIx, int offset) {
-        BSTNodeUtil.setLeftChild(frameIx, offset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
-        BSTNodeUtil.setRightChild(frameIx, offset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
-        BSTNodeUtil.setNext(frameIx, offset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
-        BSTNodeUtil.setPrev(frameIx, offset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
-    }
-
-    private ByteBuffer allocateFrame() throws HyracksDataException {
-        return ctx.allocateFrame();
-    }
-
-    public String debugPrintMemory() {
-        Slot s = new Slot(0, 0);
-        if (s.isNull()) {
-            return "memory:\tNull";
-        }
-
-        String m = "memory:\n" + debugPrintSlot(0, 0) + "\n";
-        int length = BSTNodeUtil.getActualLength(BSTNodeUtil.getLength(0, 0, frames, convertBuffer));
-        int noff = (length + 2 * BSTNodeUtil.HEADER_SIZE >= frameSize ? BSTNodeUtil.INVALID_INDEX : length + 2
-                * BSTNodeUtil.HEADER_SIZE);
-        int nfix = (noff == BSTNodeUtil.INVALID_INDEX ? ((frames.length == 1) ? BSTNodeUtil.INVALID_INDEX : 1) : 0);
-        if (noff == BSTNodeUtil.INVALID_INDEX && nfix != BSTNodeUtil.INVALID_INDEX) {
-            noff = 0;
-        }
-        s.set(nfix, noff);
-        while (!isNodeNull(s.getFrameIx(), s.getOffset())) {
-            m += debugPrintSlot(s.getFrameIx(), s.getOffset()) + "\n";
-            length = BSTNodeUtil.getActualLength(BSTNodeUtil.getLength(s.getFrameIx(), s.getOffset(), frames,
-                    convertBuffer));
-            noff = (s.getOffset() + length + 2 * BSTNodeUtil.HEADER_SIZE >= frameSize ? BSTNodeUtil.INVALID_INDEX : s
-                    .getOffset() + length + 2 * BSTNodeUtil.HEADER_SIZE);
-            nfix = (noff == BSTNodeUtil.INVALID_INDEX ? ((frames.length - 1 == s.getFrameIx()) ? BSTNodeUtil.INVALID_INDEX
-                    : s.getFrameIx() + 1)
-                    : s.getFrameIx());
-            if (noff == BSTNodeUtil.INVALID_INDEX && nfix != BSTNodeUtil.INVALID_INDEX) {
-                noff = 0;
-            }
-            s.set(nfix, noff);
-        }
-        return m;
-    }
-
-    public String debugPrintTree() {
-        Slot node = new Slot();
-        node.copy(root);
-        if (!node.isNull()) {
-            return debugPrintSubTree(node);
-        }
-        return "Null";
-    }
-
-    private String debugPrintSubTree(Slot r) {
-        Slot node = new Slot();
-        node.copy(r);
-        int fix = node.getFrameIx();
-        int off = node.getOffset();
-        int lfix = BSTNodeUtil.getLeftChildFrameIx(node, frames, convertBuffer);
-        int loff = BSTNodeUtil.getLeftChildOffset(node, frames, convertBuffer);
-        int rfix = BSTNodeUtil.getRightChildFrameIx(node, frames, convertBuffer);
-        int roff = BSTNodeUtil.getRightChildOffset(node, frames, convertBuffer);
-        int nfix = BSTNodeUtil.getNextFrameIx(node, frames, convertBuffer);
-        int noff = BSTNodeUtil.getNextOffset(node, frames, convertBuffer);
-        int pfix = BSTNodeUtil.getPrevFrameIx(node, frames, convertBuffer);
-        int poff = BSTNodeUtil.getPrevOffset(node, frames, convertBuffer);
-
-        String s = "{" + r.getFrameIx() + ", " + r.getOffset() + " (Len: "
-                + BSTNodeUtil.getLength(fix, off, frames, convertBuffer) + ") - " + "(LC: "
-                + debugPrintSlot(lfix, loff) + ") - " + "(RC: " + debugPrintSlot(rfix, roff) + ") - " + "(NX: "
-                + debugPrintSlot(nfix, noff) + ") - " + "(PR: " + debugPrintSlot(pfix, poff) + ")  }\n";
-        if (!isNodeNull(lfix, loff)) {
-            s += debugPrintSubTree(new Slot(lfix, loff)) + "\n";
-        }
-        if (!isNodeNull(rfix, roff)) {
-            s += debugPrintSubTree(new Slot(rfix, roff)) + "\n";
-        }
-
-        return s;
-    }
-
-    private String debugPrintSlot(int fix, int off) {
-        if (isNodeNull(fix, off)) {
-            return BSTNodeUtil.INVALID_INDEX + ", " + BSTNodeUtil.INVALID_INDEX;
-        }
-        int l = BSTNodeUtil.getLength(fix, off, frames, convertBuffer);
-        int al = BSTNodeUtil.getActualLength(l);
-        boolean f = BSTNodeUtil.isFree(fix, off, frames);
-        return fix + ", " + off + " (free: " + f + ") (Len: " + l + ") (actual len: " + al + ") ";
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTNodeUtil.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTNodeUtil.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTNodeUtil.java
deleted file mode 100644
index 8a1bcd3..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTNodeUtil.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-
-/**
- * @author pouria
- *         Implements utility methods, used extensively and repeatedly within
- *         the BSTMemMgr.
- *         Mainly includes methods to set/get different types of pointers,
- *         required and accessed within BST traversal, along with the methods
- *         for setting/getting length/header/footer of free slots, which have
- *         been used as the containers for BST nodes.
- */
-public class BSTNodeUtil {
-
-    static final int MINIMUM_FREE_SLOT_SIZE = 32;
-
-    private static final int FRAME_PTR_SIZE = 4;
-    private static final int OFFSET_SIZE = 2;
-
-    static final int HEADER_SIZE = 2;
-    private static final int HEADER_INDEX = 0;
-
-    private static final int LEFT_CHILD_FRAME_INDEX = HEADER_INDEX + HEADER_SIZE;
-    private static final int LEFT_CHILD_OFFSET_INDEX = LEFT_CHILD_FRAME_INDEX + FRAME_PTR_SIZE;
-
-    private static final int RIGHT_CHILD_FRAME_INDEX = LEFT_CHILD_OFFSET_INDEX + OFFSET_SIZE;
-    private static final int RIGHT_CHILD_OFFSET_INDEX = RIGHT_CHILD_FRAME_INDEX + FRAME_PTR_SIZE;
-
-    private static final int NEXT_FRAME_INDEX = RIGHT_CHILD_OFFSET_INDEX + OFFSET_SIZE;
-    private static final int NEXT_OFFSET_INDEX = NEXT_FRAME_INDEX + FRAME_PTR_SIZE;
-
-    private static final int PREV_FRAME_INDEX = NEXT_OFFSET_INDEX + OFFSET_SIZE;
-    private static final int PREV_OFFSET_INDEX = PREV_FRAME_INDEX + FRAME_PTR_SIZE;
-
-    private static final byte INVALID = -128;
-    private static final byte MASK = 127;
-    static final int INVALID_INDEX = -1;
-
-    /*
-     * Structure of a free slot:
-     * [HEADER][LEFT_CHILD][RIGHT_CHILD][NEXT][PREV]...[FOOTER] MSB in the
-     * HEADER is set to 1 in a free slot
-     * 
-     * Structure of a used slot: [HEADER]...[FOOTER] MSB in the HEADER is set to
-     * 0 in a used slot
-     */
-
-    static int getLeftChildFrameIx(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return getLeftChildFrameIx(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
-    }
-
-    static int getLeftChildOffset(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return getLeftChildOffset(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
-    }
-
-    static int getLeftChildFrameIx(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return (retrieveAsInt(frames[frameIx], offset + LEFT_CHILD_FRAME_INDEX, FRAME_PTR_SIZE, convertBuffer));
-
-    }
-
-    static int getLeftChildOffset(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return (retrieveAsInt(frames[frameIx], offset + LEFT_CHILD_OFFSET_INDEX, OFFSET_SIZE, convertBuffer));
-    }
-
-    static void setLeftChild(Slot node, Slot lc, ByteBuffer[] frames) {
-        setLeftChild(node.getFrameIx(), node.getOffset(), lc.getFrameIx(), lc.getOffset(), frames);
-    }
-
-    static void setLeftChild(int nodeFix, int nodeOff, int lcFix, int lcOff, ByteBuffer[] frames) {
-        storeInt(frames[nodeFix], nodeOff + LEFT_CHILD_FRAME_INDEX, FRAME_PTR_SIZE, lcFix);
-        storeInt(frames[nodeFix], nodeOff + LEFT_CHILD_OFFSET_INDEX, OFFSET_SIZE, lcOff);
-    }
-
-    static int getRightChildFrameIx(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return getRightChildFrameIx(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
-    }
-
-    static int getRightChildOffset(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return getRightChildOffset(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
-    }
-
-    static int getRightChildFrameIx(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return (retrieveAsInt(frames[frameIx], offset + RIGHT_CHILD_FRAME_INDEX, FRAME_PTR_SIZE, convertBuffer));
-    }
-
-    static int getRightChildOffset(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return (retrieveAsInt(frames[frameIx], offset + RIGHT_CHILD_OFFSET_INDEX, OFFSET_SIZE, convertBuffer));
-    }
-
-    static void setRightChild(Slot node, Slot rc, ByteBuffer[] frames) {
-        setRightChild(node.getFrameIx(), node.getOffset(), rc.getFrameIx(), rc.getOffset(), frames);
-    }
-
-    static void setRightChild(int nodeFix, int nodeOff, int rcFix, int rcOff, ByteBuffer[] frames) {
-        storeInt(frames[nodeFix], nodeOff + RIGHT_CHILD_FRAME_INDEX, FRAME_PTR_SIZE, rcFix);
-        storeInt(frames[nodeFix], nodeOff + RIGHT_CHILD_OFFSET_INDEX, OFFSET_SIZE, rcOff);
-    }
-
-    static int getNextFrameIx(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return getNextFrameIx(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
-    }
-
-    static int getNextOffset(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return getNextOffset(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
-    }
-
-    static int getNextFrameIx(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return (retrieveAsInt(frames[frameIx], offset + NEXT_FRAME_INDEX, FRAME_PTR_SIZE, convertBuffer));
-    }
-
-    static int getNextOffset(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return (retrieveAsInt(frames[frameIx], offset + NEXT_OFFSET_INDEX, OFFSET_SIZE, convertBuffer));
-    }
-
-    static void setNext(Slot node, Slot next, ByteBuffer[] frames) {
-        setNext(node.getFrameIx(), node.getOffset(), next.getFrameIx(), node.getOffset(), frames);
-    }
-
-    static void setNext(int nodeFix, int nodeOff, int nFix, int nOff, ByteBuffer[] frames) {
-        storeInt(frames[nodeFix], nodeOff + NEXT_FRAME_INDEX, FRAME_PTR_SIZE, nFix);
-        storeInt(frames[nodeFix], nodeOff + NEXT_OFFSET_INDEX, OFFSET_SIZE, nOff);
-    }
-
-    static int getPrevFrameIx(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return getPrevFrameIx(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
-    }
-
-    static int getPrevOffset(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return getPrevOffset(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
-    }
-
-    static int getPrevFrameIx(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return (retrieveAsInt(frames[frameIx], offset + PREV_FRAME_INDEX, FRAME_PTR_SIZE, convertBuffer));
-    }
-
-    static int getPrevOffset(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return (retrieveAsInt(frames[frameIx], offset + PREV_OFFSET_INDEX, OFFSET_SIZE, convertBuffer));
-    }
-
-    static void setPrev(Slot node, Slot prev, ByteBuffer[] frames) {
-        setPrev(node.getFrameIx(), node.getOffset(), prev.getFrameIx(), prev.getOffset(), frames);
-    }
-
-    static void setPrev(int nodeFix, int nodeOff, int pFix, int pOff, ByteBuffer[] frames) {
-        storeInt(frames[nodeFix], nodeOff + PREV_FRAME_INDEX, FRAME_PTR_SIZE, pFix);
-        storeInt(frames[nodeFix], nodeOff + PREV_OFFSET_INDEX, OFFSET_SIZE, pOff);
-    }
-
-    static boolean slotsTheSame(Slot s, Slot t) {
-        return ((s.getFrameIx() == t.getFrameIx()) && (s.getOffset() == t.getOffset()));
-    }
-
-    static void setHeaderFooter(int frameIx, int offset, int usedLength, boolean isFree, ByteBuffer[] frames) {
-        int slotLength = getActualLength(usedLength);
-        int footerOffset = offset + HEADER_SIZE + slotLength;
-        storeInt(frames[frameIx], offset, HEADER_SIZE, usedLength);
-        storeInt(frames[frameIx], footerOffset, HEADER_SIZE, usedLength);
-        setFree(frameIx, offset, isFree, frames);
-        setFree(frameIx, footerOffset, isFree, frames);
-    }
-
-    static int getLength(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        return getLength(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
-    }
-
-    static int getLength(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
-        convertBuffer.clear();
-        for (int i = 0; i < 4 - HEADER_SIZE; i++) { // padding
-            convertBuffer.put(i, (byte) 0x00);
-        }
-
-        convertBuffer.put(4 - HEADER_SIZE, (byte) ((frames[frameIx].get(offset)) & (MASK)));
-        System.arraycopy(frames[frameIx].array(), offset + 1, convertBuffer.array(), 5 - HEADER_SIZE, HEADER_SIZE - 1);
-        return convertBuffer.getInt(0);
-    }
-
-    // MSB equal to 1 means FREE
-    static boolean isFree(int frameIx, int offset, ByteBuffer[] frames) {
-        return ((((frames[frameIx]).array()[offset]) & 0x80) == 0x80);
-    }
-
-    static void setFree(int frameIx, int offset, boolean free, ByteBuffer[] frames) {
-        if (free) { // set MSB to 1 (for free)
-            frames[frameIx].put(offset, (byte) (((frames[frameIx]).array()[offset]) | 0x80));
-        } else { // set MSB to 0 (for used)
-            frames[frameIx].put(offset, (byte) (((frames[frameIx]).array()[offset]) & 0x7F));
-        }
-    }
-
-    static int getActualLength(int l) {
-        int r = (l + 2 * HEADER_SIZE) % MINIMUM_FREE_SLOT_SIZE;
-        return (r == 0 ? l : (l + (BSTNodeUtil.MINIMUM_FREE_SLOT_SIZE - r)));
-    }
-
-    private static int retrieveAsInt(ByteBuffer b, int fromIndex, int size, ByteBuffer convertBuffer) {
-        if ((b.get(fromIndex) & INVALID) == INVALID) {
-            return INVALID_INDEX;
-        }
-
-        convertBuffer.clear();
-        for (int i = 0; i < 4 - size; i++) { // padding
-            convertBuffer.put(i, (byte) 0x00);
-        }
-
-        System.arraycopy(b.array(), fromIndex, convertBuffer.array(), 4 - size, size);
-        return convertBuffer.getInt(0);
-    }
-
-    private static void storeInt(ByteBuffer b, int fromIndex, int size, int value) {
-        if (value == INVALID_INDEX) {
-            b.put(fromIndex, INVALID);
-            return;
-        }
-        for (int i = 0; i < size; i++) {
-            b.put(fromIndex + i, (byte) ((value >>> (8 * ((size - 1 - i)))) & 0xff));
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index e1315e7..3dc8b41 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -1,64 +1,42 @@
 /*
  * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
+ *  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.
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
  */
 package edu.uci.ics.hyracks.dataflow.std.sort;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.ActivityId;
-import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.TaskId;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.*;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.api.job.JobId;
-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.AbstractStateObject;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
 
-public class ExternalSortOperatorDescriptor extends AbstractOperatorDescriptor {
-    private static final long serialVersionUID = 1L;
+import java.util.List;
 
-    private static final int SORT_ACTIVITY_ID = 0;
-    private static final int MERGE_ACTIVITY_ID = 1;
+public class ExternalSortOperatorDescriptor extends AbstractSorterOperatorDescriptor {
 
-    private final int[] sortFields;
-    private final INormalizedKeyComputerFactory firstKeyNormalizerFactory;
-    private final IBinaryComparatorFactory[] comparatorFactories;
-    private final int framesLimit;
+    private static final long serialVersionUID = 1L;
 
     private Algorithm alg = Algorithm.MERGE_SORT;
+    private EnumFreeSlotPolicy policy = EnumFreeSlotPolicy.LAST_FIT;
+    private final int outputLimit;
 
     public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor, Algorithm alg) {
-        this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
-        this.alg = alg;
+        this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor, alg,
+                EnumFreeSlotPolicy.LAST_FIT);
     }
 
     public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
@@ -69,127 +47,52 @@ public class ExternalSortOperatorDescriptor extends AbstractOperatorDescriptor {
     public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDescriptor) {
-        super(spec, 1, 1);
-        this.framesLimit = framesLimit;
-        this.sortFields = sortFields;
-        this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
-        this.comparatorFactories = comparatorFactories;
-        if (framesLimit <= 1) {
-            throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
-        }
-        recordDescriptors[0] = recordDescriptor;
+        this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+                Algorithm.MERGE_SORT, EnumFreeSlotPolicy.LAST_FIT);
     }
 
     @Override
-    public void contributeActivities(IActivityGraphBuilder builder) {
-        SortActivity sa = new SortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
-        MergeActivity ma = new MergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
-
-        builder.addActivity(this, sa);
-        builder.addSourceEdge(0, sa, 0);
-
-        builder.addActivity(this, ma);
-        builder.addTargetEdge(0, ma, 0);
-
-        builder.addBlockingEdge(sa, ma);
+    public AbstractSorterOperatorDescriptor.SortActivity getSortActivity(ActivityId id) {
+        return new AbstractSorterOperatorDescriptor.SortActivity(id) {
+            @Override
+            protected AbstractSortRunGenerator getRunGenerator(IHyracksTaskContext ctx,
+                    IRecordDescriptorProvider recordDescProvider) throws HyracksDataException {
+                return new ExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory,
+                        comparatorFactories, recordDescriptors[0], alg, policy, framesLimit, outputLimit);
+            }
+        };
     }
 
-    public static class SortTaskState extends AbstractStateObject {
-        private List<IFrameReader> runs;
-        private IFrameSorter frameSorter;
-
-        public SortTaskState() {
-        }
-
-        private SortTaskState(JobId jobId, TaskId taskId) {
-            super(jobId, taskId);
-        }
-
-        @Override
-        public void toBytes(DataOutput out) throws IOException {
-
-        }
-
-        @Override
-        public void fromBytes(DataInput in) throws IOException {
-
-        }
+    @Override
+    public AbstractSorterOperatorDescriptor.MergeActivity getMergeActivity(ActivityId id) {
+        return new AbstractSorterOperatorDescriptor.MergeActivity(id) {
+            @Override
+            protected ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
+                    IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter, List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators,
+                    INormalizedKeyComputer nmkComputer, int necessaryFrames) {
+                return new ExternalSortRunMerger(ctx, sorter, runs, sortFields, comparators,
+                        nmkComputer, recordDescriptors[0], necessaryFrames, outputLimit, writer);
+            }
+        };
     }
 
-    private class SortActivity extends AbstractActivityNode {
-        private static final long serialVersionUID = 1L;
-
-        public SortActivity(ActivityId id) {
-            super(id);
-        }
-
-        @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
-            IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
-                private ExternalSortRunGenerator runGen;
-
-                @Override
-                public void open() throws HyracksDataException {
-                    runGen = new ExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory,
-                            comparatorFactories, recordDescriptors[0], alg, framesLimit);
-                    runGen.open();
-                }
-
-                @Override
-                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-                    runGen.nextFrame(buffer);
-                }
-
-                @Override
-                public void close() throws HyracksDataException {
-                    SortTaskState state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(
-                            getActivityId(), partition));
-                    runGen.close();
-                    state.runs = runGen.getRuns();
-                    state.frameSorter = runGen.getFrameSorter();
-                    ctx.setStateObject(state);
-                }
-
-                @Override
-                public void fail() throws HyracksDataException {
-                    runGen.fail();
-                }
-            };
-            return op;
-        }
+    public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, Algorithm alg, EnumFreeSlotPolicy policy) {
+        this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor, alg,
+                policy, Integer.MAX_VALUE);
     }
 
-    private class MergeActivity extends AbstractActivityNode {
-        private static final long serialVersionUID = 1L;
-
-        public MergeActivity(ActivityId id) {
-            super(id);
-        }
-
-        @Override
-        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
-            IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
-                @Override
-                public void initialize() throws HyracksDataException {
-                    SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
-                            SORT_ACTIVITY_ID), partition));
-                    List<IFrameReader> runs = state.runs;
-                    IFrameSorter frameSorter = state.frameSorter;
-                    IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
-                    for (int i = 0; i < comparatorFactories.length; ++i) {
-                        comparators[i] = comparatorFactories[i].createBinaryComparator();
-                    }
-                    INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
-                            : firstKeyNormalizerFactory.createNormalizedKeyComputer();
-                    int necessaryFrames = Math.min(runs.size() + 2, framesLimit);
-                    ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, frameSorter, runs, sortFields,
-                            comparators, nmkComputer, recordDescriptors[0], necessaryFrames, writer);
-                    merger.process();
-                }
-            };
-            return op;
+    public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, Algorithm alg, EnumFreeSlotPolicy policy, int outputLimit) {
+        super(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
+        if (framesLimit <= 1) {
+            throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
         }
+        this.alg = alg;
+        this.policy = policy;
+        this.outputLimit = outputLimit;
     }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index 3736fca..4048726 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -1,24 +1,21 @@
 /*
  * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
+ *  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.
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
  */
 package edu.uci.ics.hyracks.dataflow.std.sort;
 
 import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
 
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -27,77 +24,89 @@ import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotSmallestFit;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotBiggestFirst;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotLastFit;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameFreeSlotPolicy;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
 
-public class ExternalSortRunGenerator implements IFrameWriter {
-    private final IHyracksTaskContext ctx;
-    private final IFrameSorter frameSorter;
-    private final List<IFrameReader> runs;
-    private final int maxSortFrames;
+public class ExternalSortRunGenerator extends AbstractSortRunGenerator {
+
+    protected final IHyracksTaskContext ctx;
+    protected final IFrameSorter frameSorter;
+    protected final int maxSortFrames;
 
     public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
             RecordDescriptor recordDesc, Algorithm alg, int framesLimit) throws HyracksDataException {
+        this(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc, alg,
+                EnumFreeSlotPolicy.LAST_FIT, framesLimit);
+    }
+
+    public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDesc, Algorithm alg, EnumFreeSlotPolicy policy, int framesLimit)
+            throws HyracksDataException {
+        this(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc, alg, policy, framesLimit,
+                Integer.MAX_VALUE);
+    }
+
+    public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDesc, Algorithm alg, EnumFreeSlotPolicy policy, int framesLimit, int outputLimit)
+            throws HyracksDataException {
         this.ctx = ctx;
+        maxSortFrames = framesLimit - 1;
+
+        IFrameFreeSlotPolicy freeSlotPolicy = null;
+        switch (policy) {
+            case SMALLEST_FIT:
+                freeSlotPolicy = new FrameFreeSlotSmallestFit();
+                break;
+            case LAST_FIT:
+                freeSlotPolicy = new FrameFreeSlotLastFit(maxSortFrames);
+                break;
+            case BIGGEST_FIT:
+                freeSlotPolicy = new FrameFreeSlotBiggestFirst(maxSortFrames);
+                break;
+        }
+        IFrameBufferManager bufferManager = new VariableFrameMemoryManager(
+                new VariableFramePool(ctx, maxSortFrames * ctx.getInitialFrameSize()), freeSlotPolicy);
         if (alg == Algorithm.MERGE_SORT) {
-            frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
-                    recordDesc);
+            frameSorter = new FrameSorterMergeSort(ctx, bufferManager, sortFields, firstKeyNormalizerFactory,
+                    comparatorFactories, recordDesc, outputLimit);
         } else {
-            frameSorter = new FrameSorterQuickSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
-                    recordDesc);
+            frameSorter = new FrameSorterQuickSort(ctx, bufferManager, sortFields, firstKeyNormalizerFactory,
+                    comparatorFactories, recordDesc, outputLimit);
         }
-        runs = new LinkedList<IFrameReader>();
-        maxSortFrames = framesLimit - 1;
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        runs.clear();
-        frameSorter.reset();
     }
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        if (frameSorter.getFrameCount() >= maxSortFrames) {
+        if (!frameSorter.insertFrame(buffer)) {
             flushFramesToRun();
-        }
-        frameSorter.insertFrame(buffer);
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        if (frameSorter.getFrameCount() > 0) {
-            if (runs.size() <= 0) {
-                frameSorter.sortFrames();
-            } else {
-                flushFramesToRun();
+            if (!frameSorter.insertFrame(buffer)) {
+                throw new HyracksDataException("The given frame is too big to insert into the sorting memory.");
             }
         }
     }
 
-    private void flushFramesToRun() throws HyracksDataException {
-        frameSorter.sortFrames();
+    protected RunFileWriter getRunFileWriter() throws HyracksDataException {
         FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
                 ExternalSortRunGenerator.class.getSimpleName());
-        RunFileWriter writer = new RunFileWriter(file, ctx.getIOManager());
-        writer.open();
-        try {
-            frameSorter.flushFrames(writer);
-        } finally {
-            writer.close();
-        }
-        frameSorter.reset();
-        runs.add(writer.createReader());
+        return new RunFileWriter(file, ctx.getIOManager());
     }
 
-    @Override
-    public void fail() throws HyracksDataException {
+    protected IFrameWriter getFlushableFrameWriter(RunFileWriter writer) throws HyracksDataException {
+        return writer;
     }
 
-    public IFrameSorter getFrameSorter() {
+    @Override
+    public ISorter getSorter() {
         return frameSorter;
     }
 
-    public List<IFrameReader> getRuns() {
-        return runs;
-    }
 }
\ No newline at end of file


[11/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
index 8877df4..ccf7276 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
@@ -26,7 +26,9 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.counters.GenericCounter;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -43,7 +45,7 @@ public class ReduceWriter<K2, V2, K3, V3> implements IFrameWriter {
     private final int[] groupFields;
     private final FrameTupleAccessor accessor0;
     private final FrameTupleAccessor accessor1;
-    private final ByteBuffer copyFrame;
+    private final IFrame copyFrame;
     private final IBinaryComparator[] comparators;
     private final KVIterator kvi;
     private final Reducer<K2, V2, K3, V3> reducer;
@@ -53,7 +55,7 @@ public class ReduceWriter<K2, V2, K3, V3> implements IFrameWriter {
 
     private boolean first;
     private boolean groupStarted;
-    private List<ByteBuffer> group;
+    private List<IFrame> group;
     private int bPtr;
     private FrameTupleAppender fta;
     private Counter keyCounter;
@@ -66,10 +68,10 @@ public class ReduceWriter<K2, V2, K3, V3> implements IFrameWriter {
         this.ctx = ctx;
         this.helper = helper;
         this.groupFields = groupFields;
-        accessor0 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        accessor1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        copyFrame = ctx.allocateFrame();
-        accessor1.reset(copyFrame);
+        accessor0 = new FrameTupleAccessor(recordDescriptor);
+        accessor1 = new FrameTupleAccessor(recordDescriptor);
+        copyFrame = new VSizeFrame(ctx);
+        accessor1.reset(copyFrame.getBuffer());
         comparators = new IBinaryComparator[comparatorFactories.length];
         for (int i = 0; i < comparatorFactories.length; ++i) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -79,17 +81,17 @@ public class ReduceWriter<K2, V2, K3, V3> implements IFrameWriter {
         this.taId = taId;
         this.taskAttemptContext = taskAttemptContext;
 
-        kvi = new KVIterator(ctx, helper, recordDescriptor);
+        kvi = new KVIterator(helper, recordDescriptor);
     }
 
     @Override
     public void open() throws HyracksDataException {
         first = true;
         groupStarted = false;
-        group = new ArrayList<ByteBuffer>();
+        group = new ArrayList<>();
         bPtr = 0;
-        group.add(ctx.allocateFrame());
-        fta = new FrameTupleAppender(ctx.getFrameSize());
+        group.add(new VSizeFrame(ctx));
+        fta = new FrameTupleAppender();
         keyCounter = new GenericCounter();
         valueCounter = new GenericCounter();
     }
@@ -104,6 +106,7 @@ public class ReduceWriter<K2, V2, K3, V3> implements IFrameWriter {
                 first = false;
             } else {
                 if (i == 0) {
+                    accessor1.reset(copyFrame.getBuffer());
                     switchGroupIfRequired(accessor1, accessor1.getTupleCount() - 1, accessor0, i);
                 } else {
                     switchGroupIfRequired(accessor0, i - 1, accessor0, i);
@@ -111,20 +114,21 @@ public class ReduceWriter<K2, V2, K3, V3> implements IFrameWriter {
             }
             accumulate(accessor0, i);
         }
-        FrameUtils.copy(buffer, copyFrame);
+        copyFrame.ensureFrameSize(buffer.capacity());
+        FrameUtils.copyAndFlip(buffer, copyFrame.getBuffer());
     }
 
     private void accumulate(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
         if (!fta.append(accessor, tIndex)) {
             ++bPtr;
             if (group.size() <= bPtr) {
-                group.add(ctx.allocateFrame());
+                group.add(new VSizeFrame(ctx));
             }
             fta.reset(group.get(bPtr), true);
             if (!fta.append(accessor, tIndex)) {
                 throw new HyracksDataException("Record size ("
                         + (accessor.getTupleEndOffset(tIndex) - accessor.getTupleStartOffset(tIndex))
-                        + ") larger than frame size (" + group.get(bPtr).capacity() + ")");
+                        + ") larger than frame size (" + group.get(bPtr).getBuffer().capacity() + ")");
             }
         }
     }
@@ -137,7 +141,7 @@ public class ReduceWriter<K2, V2, K3, V3> implements IFrameWriter {
         }
     }
 
-    private void groupInit() {
+    private void groupInit() throws HyracksDataException {
         groupStarted = true;
         bPtr = 0;
         fta.reset(group.get(0), true);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
index 295fe1f..6708e17 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
@@ -22,8 +22,11 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 
-import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.NoShrinkVSizeFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -34,26 +37,29 @@ import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 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.collectors.NonDeterministicChannelReader;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
 
 public class ShuffleFrameReader implements IFrameReader {
     private final IHyracksTaskContext ctx;
     private final NonDeterministicChannelReader channelReader;
     private final HadoopHelper helper;
     private final RecordDescriptor recordDescriptor;
+    private final IFrame vframe;
     private List<RunFileWriter> runFileWriters;
+    private List<Integer> runFileMaxFrameSize;
     private RunFileReader reader;
 
     public ShuffleFrameReader(IHyracksTaskContext ctx, NonDeterministicChannelReader channelReader,
             MarshalledWritable<Configuration> mConfig) throws HyracksDataException {
         this.ctx = ctx;
         this.channelReader = channelReader;
-        helper = new HadoopHelper(mConfig);
+        this.helper = new HadoopHelper(mConfig);
         this.recordDescriptor = helper.getMapOutputRecordDescriptor();
+        this.vframe = new NoShrinkVSizeFrame(ctx);
     }
 
     @Override
@@ -61,21 +67,28 @@ public class ShuffleFrameReader implements IFrameReader {
         channelReader.open();
         int nSenders = channelReader.getSenderPartitionCount();
         runFileWriters = new ArrayList<RunFileWriter>();
+        runFileMaxFrameSize = new ArrayList<>();
         RunInfo[] infos = new RunInfo[nSenders];
-        FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-        IInputChannel[] channels = channelReader.getChannels();
+        FrameTupleAccessor accessor = new FrameTupleAccessor(recordDescriptor);
         while (true) {
             int entry = channelReader.findNextSender();
             if (entry < 0) {
                 break;
             }
             RunInfo info = infos[entry];
-            IInputChannel channel = channels[entry];
-            ByteBuffer netBuffer = channel.getNextBuffer();
-            accessor.reset(netBuffer);
+            ByteBuffer netBuffer = channelReader.getNextBuffer(entry);
+            netBuffer.clear();
+            int nBlocks = FrameHelper.deserializeNumOfMinFrame(netBuffer);
+
+            if (nBlocks > 1) {
+                netBuffer = getCompleteBuffer(nBlocks, netBuffer, entry);
+            }
+
+            accessor.reset(netBuffer, 0, netBuffer.limit());
             int nTuples = accessor.getTupleCount();
             for (int i = 0; i < nTuples; ++i) {
-                int tBlockId = IntegerPointable.getInteger(accessor.getBuffer().array(), FrameUtils.getAbsoluteFieldStartOffset(accessor, i, HadoopHelper.BLOCKID_FIELD_INDEX));
+                int tBlockId = IntegerPointable.getInteger(accessor.getBuffer().array(),
+                        accessor.getAbsoluteFieldStartOffset(i, HadoopHelper.BLOCKID_FIELD_INDEX));
                 if (info == null) {
                     info = new RunInfo();
                     info.reset(tBlockId);
@@ -86,7 +99,10 @@ public class ShuffleFrameReader implements IFrameReader {
                 }
                 info.write(accessor, i);
             }
-            channel.recycleBuffer(netBuffer);
+
+            if (nBlocks == 1) {
+                channelReader.recycleBuffer(entry, netBuffer);
+            }
         }
         for (int i = 0; i < infos.length; ++i) {
             RunInfo info = infos[i];
@@ -94,7 +110,6 @@ public class ShuffleFrameReader implements IFrameReader {
                 info.close();
             }
         }
-        infos = null;
 
         FileReference outFile = ctx.createManagedWorkspaceFile(ShuffleFrameReader.class.getName() + ".run");
         int framesLimit = helper.getSortFrameLimit(ctx);
@@ -103,22 +118,40 @@ public class ShuffleFrameReader implements IFrameReader {
         for (int i = 0; i < comparatorFactories.length; ++i) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
-        List<IFrameReader> runs = new LinkedList<IFrameReader>();
-        for (RunFileWriter rfw : runFileWriters) {
-            runs.add(rfw.createReader());
+        List<RunAndMaxFrameSizePair> runs = new LinkedList<>();
+        for (int i = 0; i < runFileWriters.size(); i++) {
+            runs.add(new RunAndMaxFrameSizePair(runFileWriters.get(i).createReader(), runFileMaxFrameSize.get(i)));
         }
         RunFileWriter rfw = new RunFileWriter(outFile, ctx.getIOManager());
-        ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, null, runs, new int[] { 0 }, comparators, null,
-                recordDescriptor, framesLimit, rfw);
+        ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, null, runs, new int[] { 0 },
+                comparators, null, recordDescriptor, framesLimit, rfw);
         merger.process();
 
         reader = rfw.createReader();
         reader.open();
     }
 
+    private ByteBuffer getCompleteBuffer(int nBlocks, ByteBuffer netBuffer, int entry) throws HyracksDataException {
+        vframe.reset();
+        vframe.ensureFrameSize(vframe.getMinSize() * nBlocks);
+        FrameUtils.copyWholeFrame(netBuffer, vframe.getBuffer());
+        channelReader.recycleBuffer(entry, netBuffer);
+        for (int i = 1; i < nBlocks; ++i) {
+            netBuffer = channelReader.getNextBuffer(entry);
+            netBuffer.clear();
+            vframe.getBuffer().put(netBuffer);
+            channelReader.recycleBuffer(entry, netBuffer);
+        }
+        if (vframe.getBuffer().hasRemaining()) { // bigger frame
+            FrameHelper.clearRemainingFrame(vframe.getBuffer(), vframe.getBuffer().position());
+        }
+        vframe.getBuffer().flip();
+        return vframe.getBuffer();
+    }
+
     @Override
-    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        return reader.nextFrame(buffer);
+    public boolean nextFrame(IFrame frame) throws HyracksDataException {
+        return reader.nextFrame(frame);
     }
 
     @Override
@@ -127,20 +160,22 @@ public class ShuffleFrameReader implements IFrameReader {
     }
 
     private class RunInfo {
-        private final ByteBuffer buffer;
+        private final IFrame buffer;
         private final FrameTupleAppender fta;
 
         private FileReference file;
         private RunFileWriter rfw;
         private int blockId;
+        private int maxFrameSize = ctx.getInitialFrameSize();
 
         public RunInfo() throws HyracksDataException {
-            buffer = ctx.allocateFrame();
-            fta = new FrameTupleAppender(ctx.getFrameSize());
+            buffer = new VSizeFrame(ctx);
+            fta = new FrameTupleAppender();
         }
 
         public void reset(int blockId) throws HyracksDataException {
             this.blockId = blockId;
+            this.maxFrameSize = ctx.getInitialFrameSize();
             fta.reset(buffer, true);
             try {
                 file = ctx.createManagedWorkspaceFile(ShuffleFrameReader.class.getName() + ".run");
@@ -165,15 +200,15 @@ public class ShuffleFrameReader implements IFrameReader {
             flush();
             rfw.close();
             runFileWriters.add(rfw);
+            runFileMaxFrameSize.add(maxFrameSize);
         }
 
         private void flush() throws HyracksDataException {
             if (fta.getTupleCount() <= 0) {
                 return;
             }
-            buffer.limit(buffer.capacity());
-            buffer.position(0);
-            rfw.nextFrame(buffer);
+            maxFrameSize = buffer.getFrameSize() > maxFrameSize ? buffer.getFrameSize() : maxFrameSize;
+            rfw.nextFrame((ByteBuffer) buffer.getBuffer().clear());
             fta.reset(buffer, true);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/pom.xml
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/pom.xml b/hyracks/hyracks-dataflow-std/pom.xml
index c4b23fd..5eadbca 100644
--- a/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks/hyracks-dataflow-std/pom.xml
@@ -66,5 +66,11 @@
   		<groupId>commons-io</groupId>
   		<artifactId>commons-io</artifactId>
   	</dependency>
+      <dependency>
+          <groupId>edu.uci.ics.hyracks</groupId>
+          <artifactId>hyracks-control-nc</artifactId>
+          <version>0.2.16-SNAPSHOT</version>
+          <scope>test</scope>
+      </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/InputChannelFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
index 607a817..12ff8e8 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
@@ -18,6 +18,8 @@ import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.channels.IInputChannel;
 import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -42,27 +44,58 @@ public class InputChannelFrameReader implements IFrameReader, IInputChannelMonit
     public void open() throws HyracksDataException {
     }
 
-    @Override
-    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        synchronized (this) {
-            while (!failed && !eos && availableFrames <= 0) {
-                try {
-                    wait();
-                } catch (InterruptedException e) {
-                    throw new HyracksDataException(e);
-                }
-            }
-            if (failed) {
-                throw new HyracksDataException("Failure occurred on input");
+    private synchronized boolean canGetNextBuffer() throws HyracksDataException {
+        while (!failed && !eos && availableFrames <= 0) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
             }
-            if (availableFrames <= 0 && eos) {
-                return false;
+        }
+        if (failed) {
+            throw new HyracksDataException("Failure occurred on input");
+        }
+        if (availableFrames <= 0 && eos) {
+            return false;
+        }
+        --availableFrames;
+        return true;
+    }
+
+    /**
+     * This implementation works under the truth that one Channel is never shared by two readers.
+     * More precisely, one channel only has exact one reader and one writer side.
+     *
+     * @param frame outputFrame
+     * @return {@code true} if succeed to read the data from the channel to the {@code frame}.
+     * Otherwise return {@code false} if the end of stream is reached.
+     * @throws HyracksDataException
+     */
+    @Override
+    public boolean nextFrame(IFrame frame) throws HyracksDataException {
+        if (!canGetNextBuffer()) {
+            return false;
+        }
+        frame.reset();
+        ByteBuffer srcFrame = channel.getNextBuffer();
+        int nBlocks = FrameHelper.deserializeNumOfMinFrame(srcFrame);
+        frame.ensureFrameSize(frame.getMinSize() * nBlocks);
+        FrameUtils.copyWholeFrame(srcFrame, frame.getBuffer());
+        channel.recycleBuffer(srcFrame);
+
+        for (int i = 1; i < nBlocks; ++i) {
+            if (!canGetNextBuffer()) {
+                throw new HyracksDataException(
+                        "InputChannelReader is waiting for the new frames, but the input stream is finished");
             }
-            --availableFrames;
+            srcFrame = channel.getNextBuffer();
+            frame.getBuffer().put(srcFrame);
+            channel.recycleBuffer(srcFrame);
+        }
+        if (frame.getBuffer().hasRemaining()) { // bigger frame
+            FrameHelper.clearRemainingFrame(frame.getBuffer(), frame.getBuffer().position());
         }
-        ByteBuffer srcBuffer = channel.getNextBuffer();
-        FrameUtils.copy(srcBuffer, buffer);
-        channel.recycleBuffer(srcBuffer);
+        frame.getBuffer().flip();
         return true;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
index 7f447c6..0c25d54 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.collectors;
 
+import java.nio.ByteBuffer;
 import java.util.BitSet;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -67,12 +68,27 @@ public class NonDeterministicChannelReader implements IInputChannelMonitor, IPar
         return nSenderPartitions;
     }
 
-    public void open() throws HyracksDataException {
-        lastReadSender = -1;
+    public synchronized ByteBuffer getNextBuffer(int index) throws HyracksDataException {
+        while ((availableFrameCounts[index] <= 0)) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        if (--availableFrameCounts[index] == 0) {
+            frameAvailability.clear(index);
+        }
+        return channels[index].getNextBuffer();
+
     }
 
-    public IInputChannel[] getChannels() {
-        return channels;
+    public void recycleBuffer(int index, ByteBuffer frame) {
+        channels[index].recycleBuffer(frame);
+    }
+
+    public void open() throws HyracksDataException {
+        lastReadSender = -1;
     }
 
     public synchronized int findNextSender() throws HyracksDataException {
@@ -83,9 +99,6 @@ public class NonDeterministicChannelReader implements IInputChannelMonitor, IPar
             }
             if (lastReadSender >= 0) {
                 assert availableFrameCounts[lastReadSender] > 0;
-                if (--availableFrameCounts[lastReadSender] == 0) {
-                    frameAvailability.clear(lastReadSender);
-                }
                 return lastReadSender;
             }
             if (!failSenders.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicFrameReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicFrameReader.java
index e107cfa..6dd6972 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicFrameReader.java
@@ -16,7 +16,8 @@ package edu.uci.ics.hyracks.dataflow.std.collectors;
 
 import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -34,16 +35,27 @@ public class NonDeterministicFrameReader implements IFrameReader {
     }
 
     @Override
-    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+    public boolean nextFrame(IFrame frame) throws HyracksDataException {
         int index = channelReader.findNextSender();
-        if (index >= 0) {
-            IInputChannel[] channels = channelReader.getChannels();
-            ByteBuffer srcFrame = channels[index].getNextBuffer();
-            FrameUtils.copy(srcFrame, buffer);
-            channels[index].recycleBuffer(srcFrame);
-            return true;
+        if (index < 0) {
+            return false;
         }
-        return false;
+        frame.reset();
+        ByteBuffer srcFrame = channelReader.getNextBuffer(index);
+        int nBlocks = FrameHelper.deserializeNumOfMinFrame(srcFrame);
+        frame.ensureFrameSize(frame.getMinSize() * nBlocks);
+        FrameUtils.copyWholeFrame(srcFrame, frame.getBuffer());
+        channelReader.recycleBuffer(index, srcFrame);
+        for (int i = 1; i < nBlocks; ++i) {
+            srcFrame = channelReader.getNextBuffer(index);
+            frame.getBuffer().put(srcFrame);
+            channelReader.recycleBuffer(index, srcFrame);
+        }
+        if (frame.getBuffer().hasRemaining()) { // bigger frame
+            FrameHelper.clearRemainingFrame(frame.getBuffer(), frame.getBuffer().position());
+        }
+        frame.getBuffer().flip();
+        return true;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
index 2dda9cc..125d07a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
@@ -14,11 +14,12 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.collectors;
 
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
@@ -54,13 +55,13 @@ public class SortMergeFrameReader implements IFrameReader {
     @Override
     public void open() throws HyracksDataException {
         if (maxConcurrentMerges >= nSenders) {
-            List<ByteBuffer> inFrames = new ArrayList<ByteBuffer>();
+            List<IFrame> inFrames = new ArrayList<>(nSenders);
             for (int i = 0; i < nSenders; ++i) {
-                inFrames.add(ByteBuffer.allocate(ctx.getFrameSize()));
+                inFrames.add(new VSizeFrame(ctx));
             }
-            List<IFrameReader> batch = new ArrayList<IFrameReader>();
+            List<IFrameReader> batch = new ArrayList<IFrameReader>(nSenders);
             pbm.getNextBatch(batch, nSenders);
-            merger = new RunMergingFrameReader(ctx, batch.toArray(new IFrameReader[nSenders]), inFrames, sortFields,
+            merger = new RunMergingFrameReader(ctx, batch, inFrames, sortFields,
                     comparators, nmkComputer, recordDescriptor);
         } else {
             // multi level merge.
@@ -70,10 +71,8 @@ public class SortMergeFrameReader implements IFrameReader {
     }
 
     @Override
-    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        buffer.position(buffer.capacity());
-        buffer.limit(buffer.capacity());
-        return merger.nextFrame(buffer);
+    public boolean nextFrame(IFrame frame) throws HyracksDataException {
+        return merger.nextFrame(frame);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
index dcca28e..edf4cc9 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
@@ -17,19 +17,22 @@ package edu.uci.ics.hyracks.dataflow.std.connectors;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 public class LocalityAwarePartitionDataWriter implements IFrameWriter {
 
     private final IFrameWriter[] pWriters;
-    private final FrameTupleAppender[] appenders;
+    private final IFrameTupleAppender[] appenders;
     private final FrameTupleAccessor tupleAccessor;
     private final ITuplePartitionComputer tpc;
 
@@ -38,17 +41,17 @@ public class LocalityAwarePartitionDataWriter implements IFrameWriter {
             ILocalityMap localityMap, int senderIndex) throws HyracksDataException {
         int[] consumerPartitions = localityMap.getConsumers(senderIndex, nConsumerPartitions);
         pWriters = new IFrameWriter[consumerPartitions.length];
-        appenders = new FrameTupleAppender[consumerPartitions.length];
+        appenders = new IFrameTupleAppender[consumerPartitions.length];
         for (int i = 0; i < consumerPartitions.length; ++i) {
             try {
                 pWriters[i] = pwFactory.createFrameWriter(consumerPartitions[i]);
-                appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
-                appenders[i].reset(ctx.allocateFrame(), true);
+                appenders[i] = new FrameTupleAppender();
+                appenders[i].reset(new VSizeFrame(ctx), true);
             } catch (IOException e) {
                 throw new HyracksDataException(e);
             }
         }
-        tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+        tupleAccessor = new FrameTupleAccessor(recordDescriptor);
         this.tpc = tpc;
     }
 
@@ -61,7 +64,6 @@ public class LocalityAwarePartitionDataWriter implements IFrameWriter {
     public void open() throws HyracksDataException {
         for (int i = 0; i < pWriters.length; ++i) {
             pWriters[i].open();
-            appenders[i].reset(appenders[i].getBuffer(), true);
         }
     }
 
@@ -77,15 +79,7 @@ public class LocalityAwarePartitionDataWriter implements IFrameWriter {
         int tupleCount = tupleAccessor.getTupleCount();
         for (int i = 0; i < tupleCount; ++i) {
             int h = pWriters.length == 1 ? 0 : tpc.partition(tupleAccessor, i, pWriters.length);
-            FrameTupleAppender appender = appenders[h];
-            if (!appender.append(tupleAccessor, i)) {
-                ByteBuffer appenderBuffer = appender.getBuffer();
-                flushFrame(appenderBuffer, pWriters[h]);
-                appender.reset(appenderBuffer, true);
-                if (!appender.append(tupleAccessor, i)) {
-                    throw new HyracksDataException("Record size (" + (tupleAccessor.getTupleEndOffset(i) - tupleAccessor.getTupleStartOffset(i)) + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
-                }
-            }
+            FrameUtils.appendToWriter(pWriters[h], appenders[h], tupleAccessor, i);
         }
     }
 
@@ -101,12 +95,6 @@ public class LocalityAwarePartitionDataWriter implements IFrameWriter {
         }
     }
 
-    private void flushFrame(ByteBuffer buffer, IFrameWriter frameWriter) throws HyracksDataException {
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
-        frameWriter.nextFrame(buffer);
-    }
-
     /*
      * (non-Javadoc)
      * 
@@ -115,9 +103,7 @@ public class LocalityAwarePartitionDataWriter implements IFrameWriter {
     @Override
     public void close() throws HyracksDataException {
         for (int i = 0; i < pWriters.length; ++i) {
-            if (appenders[i].getTupleCount() > 0) {
-                flushFrame(appenders[i].getBuffer(), pWriters[i]);
-            }
+            appenders[i].flush(pWriters[i], true);
             pWriters[i].close();
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
index ea586fc..74f16d1 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -19,12 +19,14 @@ import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 public class PartitionDataWriter implements IFrameWriter {
     private final int consumerPartitionCount;
@@ -33,7 +35,7 @@ public class PartitionDataWriter implements IFrameWriter {
     private final FrameTupleAccessor tupleAccessor;
     private final ITuplePartitionComputer tpc;
     private final IHyracksTaskContext ctx;
-    private boolean allocated = false;
+    private boolean allocatedFrame = false;
 
     public PartitionDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount, IPartitionWriterFactory pwFactory,
             RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc) throws HyracksDataException {
@@ -43,12 +45,12 @@ public class PartitionDataWriter implements IFrameWriter {
         for (int i = 0; i < consumerPartitionCount; ++i) {
             try {
                 pWriters[i] = pwFactory.createFrameWriter(i);
-                appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
+                appenders[i] = new FrameTupleAppender();
             } catch (IOException e) {
                 throw new HyracksDataException(e);
             }
         }
-        tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+        tupleAccessor = new FrameTupleAccessor(recordDescriptor);
         this.tpc = tpc;
         this.ctx = ctx;
     }
@@ -56,21 +58,13 @@ public class PartitionDataWriter implements IFrameWriter {
     @Override
     public void close() throws HyracksDataException {
         for (int i = 0; i < pWriters.length; ++i) {
-            if (allocated) {
-                if (appenders[i].getTupleCount() > 0) {
-                    flushFrame(appenders[i].getBuffer(), pWriters[i]);
-                }
+            if (allocatedFrame) {
+                appenders[i].flush(pWriters[i], true);
             }
             pWriters[i].close();
         }
     }
 
-    private void flushFrame(ByteBuffer buffer, IFrameWriter frameWriter) throws HyracksDataException {
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
-        frameWriter.nextFrame(buffer);
-    }
-
     @Override
     public void open() throws HyracksDataException {
         for (int i = 0; i < pWriters.length; ++i) {
@@ -80,34 +74,22 @@ public class PartitionDataWriter implements IFrameWriter {
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        if (!allocated) {
+        if (!allocatedFrame) {
             allocateFrames();
-            allocated = true;
+            allocatedFrame = true;
         }
         tupleAccessor.reset(buffer);
         int tupleCount = tupleAccessor.getTupleCount();
         for (int i = 0; i < tupleCount; ++i) {
             int h = tpc.partition(tupleAccessor, i, consumerPartitionCount);
-            FrameTupleAppender appender = appenders[h];
-            if (!appender.append(tupleAccessor, i)) {
-                ByteBuffer appenderBuffer = appender.getBuffer();
-                flushFrame(appenderBuffer, pWriters[h]);
-                appender.reset(appenderBuffer, true);
-                if (!appender.append(tupleAccessor, i)) {
-                    throw new HyracksDataException("Record size ("
-                            + (tupleAccessor.getTupleEndOffset(i) - tupleAccessor.getTupleStartOffset(i))
-                            + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
-                }
-            }
+            FrameUtils.appendToWriter(pWriters[h], appenders[h], tupleAccessor, i);
+
         }
     }
 
-    /**
-     * @throws HyracksDataException
-     */
     private void allocateFrames() throws HyracksDataException {
         for (int i = 0; i < appenders.length; ++i) {
-            appenders[i].reset(ctx.allocateFrame(), true);
+            appenders[i].reset(new VSizeFrame(ctx), true);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
index 5be1eab..765e223 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
@@ -18,11 +18,10 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.io.Reader;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -41,7 +40,8 @@ public class DelimitedDataTupleParserFactory implements ITupleParserFactory {
         this(fieldParserFactories, fieldDelimiter, '\"');
     }
 
-    public DelimitedDataTupleParserFactory(IValueParserFactory[] fieldParserFactories, char fieldDelimiter, char quote) {
+    public DelimitedDataTupleParserFactory(IValueParserFactory[] fieldParserFactories, char fieldDelimiter,
+            char quote) {
         this.valueParserFactories = fieldParserFactories;
         this.fieldDelimiter = fieldDelimiter;
         this.quote = quote;
@@ -57,8 +57,8 @@ public class DelimitedDataTupleParserFactory implements ITupleParserFactory {
                     for (int i = 0; i < valueParserFactories.length; ++i) {
                         valueParsers[i] = valueParserFactories[i].createValueParser();
                     }
-                    ByteBuffer frame = ctx.allocateFrame();
-                    FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+                    IFrame frame = new VSizeFrame(ctx);
+                    FrameTupleAppender appender = new FrameTupleAppender();
                     appender.reset(frame, true);
                     ArrayTupleBuilder tb = new ArrayTupleBuilder(valueParsers.length);
                     DataOutput dos = tb.getDataOutput();
@@ -80,18 +80,10 @@ public class DelimitedDataTupleParserFactory implements ITupleParserFactory {
                             valueParsers[i].parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart, dos);
                             tb.addFieldEndOffset();
                         }
-                        if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                            FrameUtils.flushFrame(frame, writer);
-                            appender.reset(frame, true);
-                            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                                throw new HyracksDataException("Record size (" + tb.getSize()
-                                        + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
-                            }
-                        }
-                    }
-                    if (appender.getTupleCount() > 0) {
-                        FrameUtils.flushFrame(frame, writer);
+                        FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+                                tb.getSize());
                     }
+                    appender.flush(writer, true);
                 } catch (IOException e) {
                     throw new HyracksDataException(e);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
index 99f5a5f..7b2e8a0 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
@@ -37,7 +37,7 @@ import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodeP
 public class PlainFileWriterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
 
     /**
-     * 
+     *
      */
     private static final long serialVersionUID = 1L;
 
@@ -45,11 +45,6 @@ public class PlainFileWriterOperatorDescriptor extends AbstractSingleActivityOpe
 
     private String delim;
 
-    /**
-     * @param spec
-     * @param inputArity
-     * @param outputArity
-     */
     public PlainFileWriterOperatorDescriptor(IOperatorDescriptorRegistry spec, IFileSplitProvider fileSplitProvider,
             String delim) {
         super(spec, 1, 0);
@@ -74,7 +69,7 @@ public class PlainFileWriterOperatorDescriptor extends AbstractSingleActivityOpe
         // Output files
         final FileSplit[] splits = fileSplitProvider.getFileSplits();
         // Frame accessor
-        final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
+        final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(
                 recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
         // Record descriptor
         final RecordDescriptor recordDescriptor = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index a4970ea..4d62fa0 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -14,11 +14,12 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.group;
 
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -33,7 +34,6 @@ 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;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 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;
@@ -76,10 +76,8 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
         }
 
         RecordDescriptor internalRecordDescriptor = outRecordDescriptor;
-        final FrameTupleAccessor storedKeysAccessor1 = new FrameTupleAccessor(ctx.getFrameSize(),
-                internalRecordDescriptor);
-        final FrameTupleAccessor storedKeysAccessor2 = new FrameTupleAccessor(ctx.getFrameSize(),
-                internalRecordDescriptor);
+        final FrameTupleAccessor storedKeysAccessor1 = new FrameTupleAccessor(internalRecordDescriptor);
+        final FrameTupleAccessor storedKeysAccessor2 = new FrameTupleAccessor(internalRecordDescriptor);
 
         final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
         for (int i = 0; i < comparatorFactories.length; ++i) {
@@ -118,14 +116,14 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
 
             private int lastBufIndex;
 
-            private ByteBuffer outputFrame;
+            private IFrame outputFrame;
             private FrameTupleAppender outputAppender;
 
-            private FrameTupleAppender stateAppender = new FrameTupleAppender(ctx.getFrameSize());
+            private FrameTupleAppender stateAppender = new FrameTupleAppender();
 
             private final ISerializableTable table = new SerializableHashTable(tableSize, ctx);
             private final TuplePointer storedTuplePointer = new TuplePointer();
-            private final List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
+            private final List<IFrame> frames = new ArrayList<>();
 
             /**
              * A tuple is "pointed" to by 3 entries in the tPointers array. [0]
@@ -153,7 +151,7 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
                         table.getTuplePointer(entry, offset, storedTuplePointer);
                         int fIndex = storedTuplePointer.frameIndex;
                         int tIndex = storedTuplePointer.tupleIndex;
-                        storedKeysAccessor1.reset(frames.get(fIndex));
+                        storedKeysAccessor1.reset(frames.get(fIndex).getBuffer());
                         int tStart = storedKeysAccessor1.getTupleStartOffset(tIndex);
                         int f0StartRel = storedKeysAccessor1.getFieldStartOffset(tIndex, sfIdx);
                         int f0EndRel = storedKeysAccessor1.getFieldEndOffset(tIndex, sfIdx);
@@ -191,7 +189,7 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
                     table.getTuplePointer(entry, offset++, storedTuplePointer);
                     if (storedTuplePointer.frameIndex < 0)
                         break;
-                    storedKeysAccessor1.reset(frames.get(storedTuplePointer.frameIndex));
+                    storedKeysAccessor1.reset(frames.get(storedTuplePointer.frameIndex).getBuffer());
                     int c = ftpcPartial.compare(accessor, tIndex, storedKeysAccessor1, storedTuplePointer.tupleIndex);
                     if (c == 0) {
                         foundGroup = true;
@@ -232,7 +230,7 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
             }
 
             @Override
-            public List<ByteBuffer> getFrames() {
+            public List<IFrame> getFrames() {
                 return frames;
             }
 
@@ -244,11 +242,11 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
             @Override
             public void flushFrames(IFrameWriter writer, boolean isPartial) throws HyracksDataException {
                 if (outputFrame == null) {
-                    outputFrame = ctx.allocateFrame();
+                    outputFrame = new VSizeFrame(ctx);
                 }
 
                 if (outputAppender == null) {
-                    outputAppender = new FrameTupleAppender(outputFrame.capacity());
+                    outputAppender = new FrameTupleAppender();
                 }
 
                 outputAppender.reset(outputFrame, true);
@@ -265,7 +263,7 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
                             int bIndex = storedTuplePointer.frameIndex;
                             int tIndex = storedTuplePointer.tupleIndex;
 
-                            storedKeysAccessor1.reset(frames.get(bIndex));
+                            storedKeysAccessor1.reset(frames.get(bIndex).getBuffer());
 
                             outputTupleBuilder.reset();
                             for (int k = 0; k < storedKeys.length; k++) {
@@ -285,8 +283,7 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
 
                             if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
                                     outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
-                                FrameUtils.flushFrame(outputFrame, writer);
-                                outputAppender.reset(outputFrame, true);
+                                outputAppender.flush(writer, true);
                                 if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
                                         outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
                                     throw new HyracksDataException(
@@ -296,10 +293,7 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
 
                         } while (true);
                     }
-                    if (outputAppender.getTupleCount() > 0) {
-                        FrameUtils.flushFrame(outputFrame, writer);
-                        outputAppender.reset(outputFrame, true);
-                    }
+                    outputAppender.flush(writer, true);
                     aggregator.close();
                     return;
                 }
@@ -311,8 +305,8 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
                     int frameIndex = storedTuplePointer.frameIndex;
                     int tupleIndex = storedTuplePointer.tupleIndex;
                     // Get the frame containing the value
-                    ByteBuffer buffer = frames.get(frameIndex);
-                    storedKeysAccessor1.reset(buffer);
+                    IFrame buffer = frames.get(frameIndex);
+                    storedKeysAccessor1.reset(buffer.getBuffer());
 
                     outputTupleBuilder.reset();
                     for (int k = 0; k < storedKeys.length; k++) {
@@ -332,18 +326,14 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
 
                     if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
                             outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
-                        FrameUtils.flushFrame(outputFrame, writer);
-                        outputAppender.reset(outputFrame, true);
+                        outputAppender.flush(writer, true);
                         if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
                                 outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
                             throw new HyracksDataException("The output item is too large to be fit into a frame.");
                         }
                     }
                 }
-                if (outputAppender.getTupleCount() > 0) {
-                    FrameUtils.flushFrame(outputFrame, writer);
-                    outputAppender.reset(outputFrame, true);
-                }
+                outputAppender.flush(writer, true);
                 aggregator.close();
             }
 
@@ -372,19 +362,14 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
 
                 if (frames.size() < framesLimit) {
                     // Insert a new frame
-                    ByteBuffer frame = ctx.allocateFrame();
-                    frame.position(0);
-                    frame.limit(frame.capacity());
+                    IFrame frame = new VSizeFrame(ctx);
                     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);
+                    stateAppender.reset(frames.get(lastBufIndex), true);
                 }
                 return true;
             }
@@ -398,7 +383,7 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
                 table.getTuplePointer(mTable, mRow, storedTuplePointer);
                 int mFrame = storedTuplePointer.frameIndex;
                 int mTuple = storedTuplePointer.tupleIndex;
-                storedKeysAccessor1.reset(frames.get(mFrame));
+                storedKeysAccessor1.reset(frames.get(mFrame).getBuffer());
 
                 int a = offset;
                 int b = a;
@@ -416,7 +401,7 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
                             table.getTuplePointer(bTable, bRow, storedTuplePointer);
                             int bFrame = storedTuplePointer.frameIndex;
                             int bTuple = storedTuplePointer.tupleIndex;
-                            storedKeysAccessor2.reset(frames.get(bFrame));
+                            storedKeysAccessor2.reset(frames.get(bFrame).getBuffer());
                             cmp = ftpcTuple.compare(storedKeysAccessor2, bTuple, storedKeysAccessor1, mTuple);
                         }
                         if (cmp > 0) {
@@ -438,7 +423,7 @@ public class HashSpillableTableFactory implements ISpillableTableFactory {
                             table.getTuplePointer(cTable, cRow, storedTuplePointer);
                             int cFrame = storedTuplePointer.frameIndex;
                             int cTuple = storedTuplePointer.tupleIndex;
-                            storedKeysAccessor2.reset(frames.get(cFrame));
+                            storedKeysAccessor2.reset(frames.get(cFrame).getBuffer());
                             cmp = ftpcTuple.compare(storedKeysAccessor2, cTuple, storedKeysAccessor1, mTuple);
                         }
                         if (cmp < 0) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
index 6ac2a6d..b0c7e4d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
@@ -14,9 +14,9 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.group;
 
-import java.nio.ByteBuffer;
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -29,7 +29,7 @@ public interface ISpillableTable {
 
     public int getFrameCount();
 
-    public List<ByteBuffer> getFrames();
+    public List<IFrame> getFrames();
 
     public void sortFrames() throws HyracksDataException;
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
index 8e9e8b8..e683e47 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
@@ -61,7 +61,7 @@ class ExternalGroupBuildOperatorNodePushable extends AbstractUnaryInputSinkOpera
         this.spillableTableFactory = spillableTableFactory;
         this.inRecordDescriptor = inRecordDescriptor;
         this.outRecordDescriptor = outRecordDescriptor;
-        this.accessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
+        this.accessor = new FrameTupleAccessor(inRecordDescriptor);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
index a55443c..9e3d4fc 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
@@ -14,14 +14,15 @@
  */
 package edu.uci.ics.hyracks.dataflow.std.group.external;
 
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.LinkedList;
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -33,7 +34,7 @@ import edu.uci.ics.hyracks.api.io.FileReference;
 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;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppenderAccessor;
 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.base.AbstractUnaryOutputSourceOperatorNodePushable;
@@ -60,12 +61,12 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
     /**
      * Input frames, one for each run file.
      */
-    private List<ByteBuffer> inFrames;
+    private List<IFrame> inFrames;
     /**
      * Output frame.
      */
-    private ByteBuffer outFrame, writerFrame;
-    private final FrameTupleAppender outAppender;
+    private IFrame outFrame, writerFrame;
+    private final FrameTupleAppenderAccessor outAppender;
     private FrameTupleAppender writerAppender;
     private LinkedList<RunFileReader> runs;
     private ExternalGroupState aggState;
@@ -76,7 +77,6 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
     private int runFrameLimit = 1;
     private int[] currentFrameIndexInRun;
     private int[] currentRunFrames;
-    private final FrameTupleAccessor outFrameAccessor;
 
     ExternalGroupMergeOperatorNodePushable(IHyracksTaskContext ctx, Object stateId,
             IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory nmkFactory, int[] keyFields,
@@ -108,8 +108,7 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
 
         tupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
         this.ctx = ctx;
-        outAppender = new FrameTupleAppender(ctx.getFrameSize());
-        outFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDescriptor);
+        outAppender = new FrameTupleAppenderAccessor(outRecordDescriptor);
         this.isOutputSorted = isOutputSorted;
         this.framesLimit = framesLimit;
         this.outRecordDescriptor = outRecordDescriptor;
@@ -132,10 +131,9 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
             } else {
                 aggState = null;
                 runs = new LinkedList<RunFileReader>(runs);
-                inFrames = new ArrayList<ByteBuffer>();
-                outFrame = ctx.allocateFrame();
+                inFrames = new ArrayList<>();
+                outFrame = new VSizeFrame(ctx);
                 outAppender.reset(outFrame, true);
-                outFrameAccessor.reset(outFrame);
                 while (runs.size() > 0) {
                     try {
                         doPass(runs);
@@ -160,7 +158,7 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
         boolean finalPass = false;
 
         while (inFrames.size() + 2 < framesLimit) {
-            inFrames.add(ctx.allocateFrame());
+            inFrames.add(new VSizeFrame(ctx));
         }
         int runNumber;
         if (runs.size() + 2 <= framesLimit) {
@@ -184,8 +182,8 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
             RunFileReader[] runFileReaders = new RunFileReader[runNumber];
             FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
             Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
-            ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), outRecordDescriptor,
-                    runNumber, comparator, keyFields, nmkComputer);
+            ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(runNumber, comparator, keyFields,
+                    nmkComputer);
             /**
              * current tuple index in each run
              */
@@ -203,8 +201,8 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
                 for (int j = 0; j < runFrameLimit; j++) {
                     int frameIndex = currentFrameIndexInRun[runIndex] + j;
                     if (runFileReaders[runIndex].nextFrame(inFrames.get(frameIndex))) {
-                        tupleAccessors[frameIndex] = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDescriptor);
-                        tupleAccessors[frameIndex].reset(inFrames.get(frameIndex));
+                        tupleAccessors[frameIndex] = new FrameTupleAccessor(outRecordDescriptor);
+                        tupleAccessors[frameIndex].reset(inFrames.get(frameIndex).getBuffer());
                         currentRunFrames[runIndex]++;
                         if (j == 0)
                             setNextTopTuple(runIndex, tupleIndices, runFileReaders, tupleAccessors, topTuples);
@@ -224,11 +222,11 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
                 ReferenceEntry top = topTuples.peek();
                 int tupleIndex = top.getTupleIndex();
                 int runIndex = topTuples.peek().getRunid();
-                FrameTupleAccessor fta = top.getAccessor();
+                IFrameTupleAccessor fta = top.getAccessor();
 
-                int currentTupleInOutFrame = outFrameAccessor.getTupleCount() - 1;
+                int currentTupleInOutFrame = outAppender.getTupleCount() - 1;
                 if (currentTupleInOutFrame < 0
-                        || compareFrameTuples(fta, tupleIndex, outFrameAccessor, currentTupleInOutFrame) != 0) {
+                        || compareFrameTuples(fta, tupleIndex, outAppender, currentTupleInOutFrame) != 0) {
                     /**
                      * Initialize the first output record Reset the
                      * tuple builder
@@ -259,7 +257,7 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
                      * outFrame
                      */
 
-                    aggregator.aggregate(fta, tupleIndex, outFrameAccessor, currentTupleInOutFrame, aggregateState);
+                    aggregator.aggregate(fta, tupleIndex, outAppender, currentTupleInOutFrame, aggregateState);
 
                 }
                 tupleIndices[runIndex]++;
@@ -295,49 +293,42 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
         }
 
         if (writerFrame == null) {
-            writerFrame = ctx.allocateFrame();
+            writerFrame = new VSizeFrame(ctx);
         }
 
         if (writerAppender == null) {
-            writerAppender = new FrameTupleAppender(ctx.getFrameSize());
+            writerAppender = new FrameTupleAppender();
             writerAppender.reset(writerFrame, true);
         }
 
-        outFrameAccessor.reset(outFrame);
-
-        for (int i = 0; i < outFrameAccessor.getTupleCount(); i++) {
+        for (int i = 0; i < outAppender.getTupleCount(); i++) {
 
             finalTupleBuilder.reset();
 
             for (int k = 0; k < storedKeys.length; k++) {
-                finalTupleBuilder.addField(outFrameAccessor, i, storedKeys[k]);
+                finalTupleBuilder.addField(outAppender, i, storedKeys[k]);
             }
 
             if (isFinal) {
 
-                aggregator.outputFinalResult(finalTupleBuilder, outFrameAccessor, i, aggregateState);
+                aggregator.outputFinalResult(finalTupleBuilder, outAppender, i, aggregateState);
 
             } else {
 
-                aggregator.outputPartialResult(finalTupleBuilder, outFrameAccessor, i, aggregateState);
+                aggregator.outputPartialResult(finalTupleBuilder, outAppender, i, aggregateState);
             }
 
             if (!writerAppender.appendSkipEmptyField(finalTupleBuilder.getFieldEndOffsets(),
                     finalTupleBuilder.getByteArray(), 0, finalTupleBuilder.getSize())) {
-                FrameUtils.flushFrame(writerFrame, writer);
-                writerAppender.reset(writerFrame, true);
+                writerAppender.flush(writer, true);
                 if (!writerAppender.appendSkipEmptyField(finalTupleBuilder.getFieldEndOffsets(),
                         finalTupleBuilder.getByteArray(), 0, finalTupleBuilder.getSize())) {
                     throw new HyracksDataException("Aggregation output is too large to be fit into a frame.");
                 }
             }
         }
-        if (writerAppender.getTupleCount() > 0) {
-            FrameUtils.flushFrame(writerFrame, writer);
-            writerAppender.reset(writerFrame, true);
-        }
+        writerAppender.flush(writer, true);
 
-        outAppender.reset(outFrame, true);
     }
 
     private void setNextTopTuple(int runIndex, int[] tupleIndices, RunFileReader[] runCursors,
@@ -377,7 +368,7 @@ class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOp
             for (int j = 0; j < runFrameLimit; j++) {
                 int frameIndex = currentFrameIndexInRun[runIndex] + j;
                 if (runCursors[runIndex].nextFrame(inFrames.get(frameIndex))) {
-                    tupleAccessors[frameIndex].reset(inFrames.get(frameIndex));
+                    tupleAccessors[frameIndex].reset(inFrames.get(frameIndex).getBuffer());
                     existNext = true;
                     currentRunFrames[runIndex]++;
                 } else {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
index 3c0eb2b..0102e65 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
@@ -19,7 +19,9 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -32,6 +34,7 @@ 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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 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;
@@ -40,7 +43,7 @@ class GroupingHashTable {
     /**
      * The pointers in the link store 3 int values for each entry in the
      * hashtable: (bufferIdx, tIndex, accumulatorIdx).
-     * 
+     *
      * @author vinayakb
      */
     private static class Link {
@@ -67,7 +70,7 @@ class GroupingHashTable {
     private static final int INIT_AGG_STATE_SIZE = 8;
     private final IHyracksTaskContext ctx;
 
-    private final List<ByteBuffer> buffers;
+    private final List<IFrame> buffers;
     private final Link[] table;
     /**
      * Aggregate states: a list of states for all groups maintained in the main
@@ -84,6 +87,7 @@ class GroupingHashTable {
     private final ITuplePartitionComputer tpc;
     private final IAggregatorDescriptor aggregator;
 
+    private final IFrame outputFrame;
     private final FrameTupleAppender appender;
 
     private final FrameTupleAccessor storedKeysAccessor;
@@ -96,7 +100,7 @@ class GroupingHashTable {
             throws HyracksDataException {
         this.ctx = ctx;
 
-        buffers = new ArrayList<ByteBuffer>();
+        buffers = new ArrayList<>();
         table = new Link[tableSize];
 
         keys = fields;
@@ -127,10 +131,10 @@ class GroupingHashTable {
         accumulatorSize = 0;
 
         RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(storedKeySerDeser);
-        storedKeysAccessor = new FrameTupleAccessor(ctx.getFrameSize(), storedKeysRecordDescriptor);
+        storedKeysAccessor = new FrameTupleAccessor(storedKeysRecordDescriptor);
         lastBIndex = -1;
 
-        appender = new FrameTupleAppender(ctx.getFrameSize());
+        appender = new FrameTupleAppender();
 
         addNewBuffer();
 
@@ -140,14 +144,13 @@ class GroupingHashTable {
             stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length + 1);
         }
         outputTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+        outputFrame = new VSizeFrame(ctx);
     }
 
     private void addNewBuffer() throws HyracksDataException {
-        ByteBuffer buffer = ctx.allocateFrame();
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
-        buffers.add(buffer);
-        appender.reset(buffer, true);
+        VSizeFrame frame = new VSizeFrame(ctx);
+        buffers.add(frame);
+        appender.reset(frame, true);
         ++lastBIndex;
     }
 
@@ -161,7 +164,7 @@ class GroupingHashTable {
         for (int i = 0; i < link.size; i += 3) {
             int sbIndex = link.pointers[i];
             int stIndex = link.pointers[i + 1];
-            storedKeysAccessor.reset(buffers.get(sbIndex));
+            storedKeysAccessor.reset(buffers.get(sbIndex).getBuffer());
             int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
             if (c == 0) {
                 saIndex = link.pointers[i + 2];
@@ -206,8 +209,7 @@ class GroupingHashTable {
     }
 
     void write(IFrameWriter writer) throws HyracksDataException {
-        ByteBuffer buffer = ctx.allocateFrame();
-        appender.reset(buffer, true);
+        appender.reset(outputFrame, true);
 
         for (int i = 0; i < table.length; ++i) {
             Link link = table[i];
@@ -216,7 +218,7 @@ class GroupingHashTable {
                     int bIndex = link.pointers[j];
                     int tIndex = link.pointers[j + 1];
                     int aIndex = link.pointers[j + 2];
-                    ByteBuffer keyBuffer = buffers.get(bIndex);
+                    ByteBuffer keyBuffer = buffers.get(bIndex).getBuffer();
                     storedKeysAccessor.reset(keyBuffer);
 
                     // copy keys
@@ -228,22 +230,13 @@ class GroupingHashTable {
                     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.");
-                        }
-                    }
+                    FrameUtils.appendSkipEmptyFieldToWriter(writer, appender, outputTupleBuilder.getFieldEndOffsets(),
+                            outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize());
 
                 }
             }
         }
-        if (appender.getTupleCount() != 0) {
-            writer.nextFrame(buffer);
-        }
+        appender.flush(writer, true);
     }
 
     void close() throws HyracksDataException {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java
index 8e49a9a..998d882 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java
@@ -44,7 +44,7 @@ class HashGroupBuildOperatorNodePushable extends AbstractUnaryInputSinkOperatorN
             IAggregatorDescriptorFactory aggregatorFactory, int tableSize, RecordDescriptor inRecordDescriptor,
             RecordDescriptor outRecordDescriptor) {
         this.ctx = ctx;
-        this.accessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
+        this.accessor = new FrameTupleAccessor(inRecordDescriptor);
         this.stateId = stateId;
         this.keys = keys;
         this.tpcf = tpcf;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
index 9ce70c1..4dbf03b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
@@ -21,8 +21,6 @@ import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 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.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
 
@@ -52,12 +50,6 @@ class PreclusteredGroupOperatorNodePushable extends AbstractUnaryInputUnaryOutpu
         for (int i = 0; i < comparatorFactories.length; ++i) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
-        final ByteBuffer copyFrame = ctx.allocateFrame();
-        final FrameTupleAccessor copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
-        copyFrameAccessor.reset(copyFrame);
-        ByteBuffer outFrame = ctx.allocateFrame();
-        final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-        appender.reset(outFrame, true);
         pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregatorFactory, inRecordDescriptor,
                 outRecordDescriptor, writer);
         pgw.open();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
index 45f0488..559dec4 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
@@ -16,15 +16,17 @@ package edu.uci.ics.hyracks.dataflow.std.group.preclustered;
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.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.FrameTupleAppenderWrapper;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
 import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
@@ -35,7 +37,7 @@ public class PreclusteredGroupWriter implements IFrameWriter {
     private final IBinaryComparator[] comparators;
     private final IAggregatorDescriptor aggregator;
     private final AggregateState aggregateState;
-    private final ByteBuffer copyFrame;
+    private final IFrame copyFrame;
     private final FrameTupleAccessor inFrameAccessor;
     private final FrameTupleAccessor copyFrameAccessor;
 
@@ -62,15 +64,15 @@ public class PreclusteredGroupWriter implements IFrameWriter {
         this.aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc, outRecordDesc, groupFields,
                 groupFields, writer);
         this.aggregateState = aggregator.createAggregateStates();
-        copyFrame = ctx.allocateFrame();
-        inFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
-        copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
-        copyFrameAccessor.reset(copyFrame);
+        copyFrame = new VSizeFrame(ctx);
+        inFrameAccessor = new FrameTupleAccessor(inRecordDesc);
+        copyFrameAccessor = new FrameTupleAccessor(inRecordDesc);
+        copyFrameAccessor.reset(copyFrame.getBuffer());
 
-        ByteBuffer outFrame = ctx.allocateFrame();
-        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        VSizeFrame outFrame = new VSizeFrame(ctx);
+        FrameTupleAppender appender = new FrameTupleAppender();
         appender.reset(outFrame, true);
-        appenderWrapper = new FrameTupleAppenderWrapper(appender, outFrame, writer);
+        appenderWrapper = new FrameTupleAppenderWrapper(appender, writer);
 
         tupleBuilder = new ArrayTupleBuilder(outRecordDesc.getFields().length);
     }
@@ -105,7 +107,9 @@ public class PreclusteredGroupWriter implements IFrameWriter {
 
             }
         }
-        FrameUtils.copy(buffer, copyFrame);
+        copyFrame.ensureFrameSize(buffer.capacity());
+        FrameUtils.copyAndFlip(buffer, copyFrame.getBuffer());
+        copyFrameAccessor.reset(copyFrame.getBuffer());
     }
 
     private void switchGroupIfRequired(FrameTupleAccessor prevTupleAccessor, int prevTupleIndex,
@@ -145,9 +149,9 @@ public class PreclusteredGroupWriter implements IFrameWriter {
             throws HyracksDataException {
         for (int i = 0; i < comparators.length; ++i) {
             int fIdx = groupFields[i];
-            int s1 = a1.getTupleStartOffset(t1Idx) + a1.getFieldSlotsLength() + a1.getFieldStartOffset(t1Idx, fIdx);
+            int s1 = a1.getAbsoluteFieldStartOffset(t1Idx, fIdx);
             int l1 = a1.getFieldLength(t1Idx, fIdx);
-            int s2 = a2.getTupleStartOffset(t2Idx) + a2.getFieldSlotsLength() + a2.getFieldStartOffset(t2Idx, fIdx);
+            int s2 = a2.getAbsoluteFieldStartOffset(t2Idx, fIdx);
             int l2 = a2.getFieldLength(t2Idx, fIdx);
             if (comparators[i].compare(a1.getBuffer().array(), s1, l1, a2.getBuffer().array(), s2, l2) != 0) {
                 return false;
@@ -165,6 +169,7 @@ public class PreclusteredGroupWriter implements IFrameWriter {
     @Override
     public void close() throws HyracksDataException {
         if (!isFailed && !first) {
+            assert(copyFrameAccessor.getTupleCount() > 0);
             writeOutput(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1);
             appenderWrapper.flush();
         }


[12/14] incubator-asterixdb-hyracks git commit: VariableSizeFrame(VSizeFrame) support for Hyracks.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
index 98b8e79..8a3c35d 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
@@ -1,164 +1,121 @@
 /*
  * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
+ *  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.
+ *
+ *  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.common.comm.io;
 
 import java.nio.ByteBuffer;
 
 import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
 
-public class FrameTupleAppender {
-    private final int frameSize;
+public class FrameTupleAppender extends AbstractFrameAppender implements IFrameTupleAppender {
 
-    private ByteBuffer buffer;
-
-    private int tupleCount;
-
-    private int tupleDataEndOffset;
-
-    private int numberOfFields = -1;
-    private int currentField = 0;
-    private int lastFieldEndOffset = 0;
-
-    public FrameTupleAppender(int frameSize) {
-        this.frameSize = frameSize;
+    public FrameTupleAppender() {
     }
 
-    public FrameTupleAppender(int frameSize, int numberOfFields) {
-        this.frameSize = frameSize;
-        this.numberOfFields = numberOfFields;
+    public FrameTupleAppender(IFrame frame) throws HyracksDataException {
+        reset(frame, true);
     }
 
-    public void reset(ByteBuffer buffer, boolean clear) {
-        this.buffer = buffer;
-        if (clear) {
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), 0);
-            tupleCount = 0;
-            tupleDataEndOffset = 0;
-        } else {
-            tupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
-            tupleDataEndOffset = tupleCount == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize)
-                    - tupleCount * 4);
-        }
+    public FrameTupleAppender(IFrame frame, boolean clear) throws HyracksDataException {
+        reset(frame, clear);
     }
 
-    public boolean append(int[] fieldSlots, byte[] bytes, int offset, int length) {
-        if (tupleDataEndOffset + fieldSlots.length * 4 + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
+    public boolean append(int[] fieldSlots, byte[] bytes, int offset, int length) throws HyracksDataException {
+        if (canHoldNewTuple(fieldSlots.length, length)) {
             for (int i = 0; i < fieldSlots.length; ++i) {
-                buffer.putInt(tupleDataEndOffset + i * 4, fieldSlots[i]);
+                IntSerDeUtils.putInt(array, tupleDataEndOffset + i * 4, fieldSlots[i]);
             }
-            System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset + fieldSlots.length * 4, length);
+            System.arraycopy(bytes, offset, array, tupleDataEndOffset + fieldSlots.length * 4, length);
             tupleDataEndOffset += fieldSlots.length * 4 + length;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            IntSerDeUtils.putInt(getBuffer().array(),
+                    FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+            IntSerDeUtils
+                    .putInt(getBuffer().array(), FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
             return true;
         }
         return false;
     }
 
-    public boolean append(byte[] bytes, int offset, int length) {
-        if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
-            System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset, length);
+    public boolean append(byte[] bytes, int offset, int length) throws HyracksDataException {
+        if (canHoldNewTuple(0, length)) {
+            System.arraycopy(bytes, offset, getBuffer().array(), tupleDataEndOffset, length);
             tupleDataEndOffset += length;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            IntSerDeUtils.putInt(array,
+                    FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+            IntSerDeUtils
+                    .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
             return true;
         }
         return false;
     }
 
-    public boolean appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length) {
-        if (tupleDataEndOffset + fieldSlots.length * 4 + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
+    public boolean appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length)
+            throws HyracksDataException {
+        if (canHoldNewTuple(fieldSlots.length, length)) {
             int effectiveSlots = 0;
             for (int i = 0; i < fieldSlots.length; ++i) {
                 if (fieldSlots[i] > 0) {
-                    buffer.putInt(tupleDataEndOffset + i * 4, fieldSlots[i]);
+                    IntSerDeUtils.putInt(array, tupleDataEndOffset + i * 4, fieldSlots[i]);
                     effectiveSlots++;
                 }
             }
-            System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset + effectiveSlots * 4, length);
+            System.arraycopy(bytes, offset, array, tupleDataEndOffset + effectiveSlots * 4, length);
             tupleDataEndOffset += effectiveSlots * 4 + length;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            IntSerDeUtils.putInt(array,
+                    FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1),
+                    tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+            IntSerDeUtils
+                    .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
             return true;
         }
         return false;
     }
 
-    public boolean appendField(byte[] bytes, int offset, int length) {
-        if (numberOfFields < 0) {
-            throw new IllegalStateException("unintialized number of fields " + numberOfFields);
-        }
-        int currentTupleDataStart = tupleDataEndOffset + numberOfFields * 4 + lastFieldEndOffset;
-        if (currentTupleDataStart + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
-            System.arraycopy(bytes, offset, buffer.array(), currentTupleDataStart, length);
-            lastFieldEndOffset = lastFieldEndOffset + length;
-            buffer.putInt(tupleDataEndOffset + currentField * 4, lastFieldEndOffset);
-            if (++currentField == numberOfFields) {
-                tupleDataEndOffset += numberOfFields * 4 + lastFieldEndOffset;
-                buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
-                ++tupleCount;
-                buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
-
-                //reset for the next tuple
-                currentField = 0;
-                lastFieldEndOffset = 0;
-            }
-            return true;
-        } else {
-            //reset for the next tuple
-            currentField = 0;
-            lastFieldEndOffset = 0;
-            return false;
-        }
-    }
-
-    public boolean appendField(IFrameTupleAccessor fta, int tIndex, int fIndex) {
-        if (numberOfFields < 0) {
-            throw new IllegalStateException("unintialized number of fields " + numberOfFields);
-        }
-        int startOffset = fta.getTupleStartOffset(tIndex);
-        int fStartOffset = fta.getFieldStartOffset(tIndex, fIndex);
-        int fLen = fta.getFieldEndOffset(tIndex, fIndex) - fStartOffset;
-        return appendField(fta.getBuffer().array(), startOffset + fta.getFieldSlotsLength() + fStartOffset, fLen);
-    }
-
-    public boolean append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset) {
+    public boolean append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset)
+            throws HyracksDataException {
         int length = tEndOffset - tStartOffset;
-        if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
+        if (canHoldNewTuple(0, length)) {
             ByteBuffer src = tupleAccessor.getBuffer();
-            System.arraycopy(src.array(), tStartOffset, buffer.array(), tupleDataEndOffset, length);
+            System.arraycopy(src.array(), tStartOffset, array, tupleDataEndOffset, length);
             tupleDataEndOffset += length;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            IntSerDeUtils.putInt(array,
+                    FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1),
+                    tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+            IntSerDeUtils
+                    .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
             return true;
         }
         return false;
     }
 
-    public boolean append(IFrameTupleAccessor tupleAccessor, int tIndex) {
+    public boolean append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException {
         int tStartOffset = tupleAccessor.getTupleStartOffset(tIndex);
         int tEndOffset = tupleAccessor.getTupleEndOffset(tIndex);
         return append(tupleAccessor, tStartOffset, tEndOffset);
     }
 
-    public boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1) {
+    public boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1,
+            int tIndex1) throws HyracksDataException {
         int startOffset0 = accessor0.getTupleStartOffset(tIndex0);
         int endOffset0 = accessor0.getTupleEndOffset(tIndex0);
         int length0 = endOffset0 - startOffset0;
@@ -167,7 +124,7 @@ public class FrameTupleAppender {
         int endOffset1 = accessor1.getTupleEndOffset(tIndex1);
         int length1 = endOffset1 - startOffset1;
 
-        if (tupleDataEndOffset + length0 + length1 + 4 + (tupleCount + 1) * 4 <= frameSize) {
+        if (canHoldNewTuple(0, length0 + length1)) {
             ByteBuffer src0 = accessor0.getBuffer();
             ByteBuffer src1 = accessor1.getBuffer();
             int slotsLen0 = accessor0.getFieldSlotsLength();
@@ -175,28 +132,31 @@ public class FrameTupleAppender {
             int dataLen0 = length0 - slotsLen0;
             int dataLen1 = length1 - slotsLen1;
             // Copy slots from accessor0 verbatim
-            System.arraycopy(src0.array(), startOffset0, buffer.array(), tupleDataEndOffset, slotsLen0);
+            System.arraycopy(src0.array(), startOffset0, array, tupleDataEndOffset, slotsLen0);
             // Copy slots from accessor1 with the following transformation: newSlotIdx = oldSlotIdx + dataLen0
             for (int i = 0; i < slotsLen1 / 4; ++i) {
-                buffer.putInt(tupleDataEndOffset + slotsLen0 + i * 4, src1.getInt(startOffset1 + i * 4) + dataLen0);
+                IntSerDeUtils.putInt(array, tupleDataEndOffset + slotsLen0 + i * 4,
+                        src1.getInt(startOffset1 + i * 4) + dataLen0);
             }
             // Copy data0
-            System.arraycopy(src0.array(), startOffset0 + slotsLen0, buffer.array(), tupleDataEndOffset + slotsLen0
+            System.arraycopy(src0.array(), startOffset0 + slotsLen0, array, tupleDataEndOffset + slotsLen0
                     + slotsLen1, dataLen0);
             // Copy data1
-            System.arraycopy(src1.array(), startOffset1 + slotsLen1, buffer.array(), tupleDataEndOffset + slotsLen0
+            System.arraycopy(src1.array(), startOffset1 + slotsLen1, array, tupleDataEndOffset + slotsLen0
                     + slotsLen1 + dataLen0, dataLen1);
             tupleDataEndOffset += (length0 + length1);
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            IntSerDeUtils.putInt(array,
+                    FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+            IntSerDeUtils
+                    .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
             return true;
         }
         return false;
     }
 
     public boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, int[] fieldSlots1, byte[] bytes1,
-            int offset1, int dataLen1) {
+            int offset1, int dataLen1) throws HyracksDataException {
         int startOffset0 = accessor0.getTupleStartOffset(tIndex0);
         int endOffset0 = accessor0.getTupleEndOffset(tIndex0);
         int length0 = endOffset0 - startOffset0;
@@ -204,33 +164,36 @@ public class FrameTupleAppender {
         int slotsLen1 = fieldSlots1.length * 4;
         int length1 = slotsLen1 + dataLen1;
 
-        if (tupleDataEndOffset + length0 + length1 + 4 + (tupleCount + 1) * 4 <= frameSize) {
+        if (canHoldNewTuple(0, length0 + length1)) {
             ByteBuffer src0 = accessor0.getBuffer();
             int slotsLen0 = accessor0.getFieldSlotsLength();
             int dataLen0 = length0 - slotsLen0;
             // Copy slots from accessor0 verbatim
-            System.arraycopy(src0.array(), startOffset0, buffer.array(), tupleDataEndOffset, slotsLen0);
+            System.arraycopy(src0.array(), startOffset0, array, tupleDataEndOffset, slotsLen0);
             // Copy fieldSlots1 with the following transformation: newSlotIdx = oldSlotIdx + dataLen0
             for (int i = 0; i < fieldSlots1.length; ++i) {
-                buffer.putInt(tupleDataEndOffset + slotsLen0 + i * 4, (fieldSlots1[i] + dataLen0));
+                IntSerDeUtils.putInt(array, tupleDataEndOffset + slotsLen0 + i * 4,
+                        (fieldSlots1[i] + dataLen0));
             }
             // Copy data0
-            System.arraycopy(src0.array(), startOffset0 + slotsLen0, buffer.array(), tupleDataEndOffset + slotsLen0
+            System.arraycopy(src0.array(), startOffset0 + slotsLen0, array, tupleDataEndOffset + slotsLen0
                     + slotsLen1, dataLen0);
             // Copy bytes1
-            System.arraycopy(bytes1, offset1, buffer.array(), tupleDataEndOffset + slotsLen0 + fieldSlots1.length * 4
-                    + dataLen0, dataLen1);
+            System.arraycopy(bytes1, offset1, array,
+                    tupleDataEndOffset + slotsLen0 + fieldSlots1.length * 4 + dataLen0, dataLen1);
             tupleDataEndOffset += (length0 + length1);
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            IntSerDeUtils.putInt(array,
+                    FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+            IntSerDeUtils
+                    .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
             return true;
         }
         return false;
     }
 
     public boolean appendConcat(int[] fieldSlots0, byte[] bytes0, int offset0, int dataLen0,
-            IFrameTupleAccessor accessor1, int tIndex1) {
+            IFrameTupleAccessor accessor1, int tIndex1) throws HyracksDataException {
         int slotsLen0 = fieldSlots0.length * 4;
         int length0 = slotsLen0 + dataLen0;
 
@@ -238,40 +201,45 @@ public class FrameTupleAppender {
         int endOffset1 = accessor1.getTupleEndOffset(tIndex1);
         int length1 = endOffset1 - startOffset1;
 
-        if (tupleDataEndOffset + length0 + length1 + 4 + (tupleCount + 1) * 4 <= frameSize) {
+        if (canHoldNewTuple(0, length0 + length1)) {
             ByteBuffer src1 = accessor1.getBuffer();
             int slotsLen1 = accessor1.getFieldSlotsLength();
             int dataLen1 = length1 - slotsLen1;
             // Copy fieldSlots0 verbatim
             for (int i = 0; i < fieldSlots0.length; ++i) {
-                buffer.putInt(tupleDataEndOffset + i * 4, fieldSlots0[i]);
+                IntSerDeUtils.putInt(array, tupleDataEndOffset + i * 4, fieldSlots0[i]);
             }
             // Copy slots from accessor1 with the following transformation: newSlotIdx = oldSlotIdx + dataLen0
             for (int i = 0; i < slotsLen1 / 4; ++i) {
-                buffer.putInt(tupleDataEndOffset + slotsLen0 + i * 4, src1.getInt(startOffset1 + i * 4) + dataLen0);
+                IntSerDeUtils.putInt(array, tupleDataEndOffset + slotsLen0 + i * 4,
+                        src1.getInt(startOffset1 + i * 4) + dataLen0);
             }
             // Copy bytes0
-            System.arraycopy(bytes0, offset0, buffer.array(), tupleDataEndOffset + slotsLen0 + slotsLen1, dataLen0);
+            System.arraycopy(bytes0, offset0, array, tupleDataEndOffset + slotsLen0 + slotsLen1,
+                    dataLen0);
             // Copy data1
-            System.arraycopy(src1.array(), startOffset1 + slotsLen1, buffer.array(), tupleDataEndOffset + slotsLen0
+            System.arraycopy(src1.array(), startOffset1 + slotsLen1, array, tupleDataEndOffset + slotsLen0
                     + slotsLen1 + dataLen0, dataLen1);
             tupleDataEndOffset += (length0 + length1);
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            IntSerDeUtils.putInt(array,
+                    FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+            IntSerDeUtils
+                    .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
             return true;
         }
         return false;
     }
 
-    public boolean appendProjection(IFrameTupleAccessor accessor, int tIndex, int[] fields) {
+    public boolean appendProjection(IFrameTupleAccessor accessor, int tIndex, int[] fields)
+            throws HyracksDataException {
         int fTargetSlotsLength = fields.length * 4;
         int length = fTargetSlotsLength;
         for (int i = 0; i < fields.length; ++i) {
             length += (accessor.getFieldEndOffset(tIndex, fields[i]) - accessor.getFieldStartOffset(tIndex, fields[i]));
         }
 
-        if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
+        if (canHoldNewTuple(0, length)) {
             int fSrcSlotsLength = accessor.getFieldSlotsLength();
             int tStartOffset = accessor.getTupleStartOffset(tIndex);
 
@@ -281,26 +249,21 @@ public class FrameTupleAppender {
                 int fSrcStart = tStartOffset + fSrcSlotsLength + accessor.getFieldStartOffset(tIndex, fields[i]);
                 int fLen = accessor.getFieldEndOffset(tIndex, fields[i])
                         - accessor.getFieldStartOffset(tIndex, fields[i]);
-                System.arraycopy(accessor.getBuffer().array(), fSrcStart, buffer.array(), tupleDataEndOffset
+                System.arraycopy(accessor.getBuffer().array(), fSrcStart, array, tupleDataEndOffset
                         + fTargetSlotsLength + fStartOffset, fLen);
                 fEndOffset += fLen;
-                buffer.putInt(tupleDataEndOffset + i * 4, fEndOffset);
+                IntSerDeUtils.putInt(array, tupleDataEndOffset + i * 4, fEndOffset);
                 fStartOffset = fEndOffset;
             }
             tupleDataEndOffset += length;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+            IntSerDeUtils.putInt(array,
+                    FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
             ++tupleCount;
-            buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+            IntSerDeUtils
+                    .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
             return true;
         }
         return false;
     }
 
-    public int getTupleCount() {
-        return tupleCount;
-    }
-
-    public ByteBuffer getBuffer() {
-        return buffer;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderAccessor.java
new file mode 100644
index 0000000..b29c8d1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderAccessor.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+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.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+
+public class FrameTupleAppenderAccessor extends FrameTupleAppender implements IFrameTupleAccessor {
+    private int tupleCountOffset;
+    private final RecordDescriptor recordDescriptor;
+
+    public FrameTupleAppenderAccessor(RecordDescriptor recordDescriptor) {
+        super();
+        this.recordDescriptor = recordDescriptor;
+    }
+
+    @Override
+    public void reset(ByteBuffer buffer) {
+        throw new IllegalAccessError("should not call this function");
+    }
+
+    @Override
+    public int getTupleStartOffset(int tupleIndex) {
+        int offset = tupleIndex == 0 ?
+                FrameConstants.TUPLE_START_OFFSET :
+                IntSerDeUtils.getInt(getBuffer().array(), tupleCountOffset - 4 * tupleIndex);
+        return offset;
+    }
+
+    @Override
+    public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+        return getTupleStartOffset(tupleIndex) + getFieldSlotsLength() + getFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
+    public int getTupleEndOffset(int tupleIndex) {
+        return IntSerDeUtils.getInt(getBuffer().array(), tupleCountOffset - 4 * (tupleIndex + 1));
+    }
+
+    @Override
+    public int getFieldStartOffset(int tupleIndex, int fIdx) {
+        return fIdx == 0 ? 0 : IntSerDeUtils.getInt(getBuffer().array(),
+                getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+    }
+
+    @Override
+    public int getFieldEndOffset(int tupleIndex, int fIdx) {
+        return IntSerDeUtils.getInt(getBuffer().array(), getTupleStartOffset(tupleIndex) + fIdx * 4);
+    }
+
+    @Override
+    public int getFieldLength(int tupleIndex, int fIdx) {
+        return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
+    public int getTupleLength(int tupleIndex) {
+        return getTupleEndOffset(tupleIndex) - getTupleStartOffset(tupleIndex);
+    }
+
+    @Override
+    public int getFieldSlotsLength() {
+        return getFieldCount() * 4;
+    }
+
+    public void prettyPrint() {
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+        DataInputStream dis = new DataInputStream(bbis);
+        int tc = getTupleCount();
+        System.err.println("TC: " + tc);
+        for (int i = 0; i < tc; ++i) {
+            prettyPrint(i, bbis, dis);
+        }
+    }
+
+    protected void prettyPrint(int tid, ByteBufferInputStream bbis, DataInputStream dis) {
+        System.err.print("tid" + tid + ":(" + getTupleStartOffset(tid) + ", " + getTupleEndOffset(tid) + ")[");
+        for (int j = 0; j < getFieldCount(); ++j) {
+            System.err.print("f" + j + ":(" + getFieldStartOffset(tid, j) + ", " + getFieldEndOffset(tid, j) + ") ");
+            System.err.print("{");
+            bbis.setByteBuffer(getBuffer(),
+                    getTupleStartOffset(tid) + getFieldSlotsLength() + getFieldStartOffset(tid, j));
+            try {
+                System.err.print(recordDescriptor.getFields()[j].deserialize(dis));
+            } catch (HyracksDataException e) {
+                e.printStackTrace();
+            }
+            System.err.print("}");
+        }
+        System.err.println();
+    }
+
+    public void prettyPrint(int tid) {
+        ByteBufferInputStream bbis = new ByteBufferInputStream();
+        DataInputStream dis = new DataInputStream(bbis);
+        prettyPrint(tid, bbis, dis);
+    }
+
+    @Override
+    public int getFieldCount() {
+        return recordDescriptor.getFieldCount();
+    }
+
+    @Override
+    public void reset(IFrame frame, boolean clear) throws HyracksDataException {
+        super.reset(frame, clear);
+        this.tupleCountOffset = FrameHelper.getTupleCountOffset(frame.getFrameSize());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderWrapper.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderWrapper.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderWrapper.java
index 2de4256..b8e7450 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderWrapper.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderWrapper.java
@@ -15,9 +15,9 @@
 
 package edu.uci.ics.hyracks.dataflow.common.comm.io;
 
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -26,18 +26,15 @@ import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
  * This class wraps the calls of FrameTupleAppender and
  * allows user to not worry about flushing full frames.
  * TODO(yingyib): cleanup existing usage of FrameTupleAppender.
- * 
+ *
  * @author yingyib
  */
 public class FrameTupleAppenderWrapper {
-    private final FrameTupleAppender frameTupleAppender;
-    private final ByteBuffer outputFrame;
+    private final IFrameTupleAppender frameTupleAppender;
     private final IFrameWriter outputWriter;
 
-    public FrameTupleAppenderWrapper(FrameTupleAppender frameTupleAppender, ByteBuffer outputFrame,
-            IFrameWriter outputWriter) {
+    public FrameTupleAppenderWrapper(IFrameTupleAppender frameTupleAppender, IFrameWriter outputWriter) {
         this.frameTupleAppender = frameTupleAppender;
-        this.outputFrame = outputFrame;
         this.outputWriter = outputWriter;
     }
 
@@ -46,9 +43,7 @@ public class FrameTupleAppenderWrapper {
     }
 
     public void flush() throws HyracksDataException {
-        if (frameTupleAppender.getTupleCount() > 0) {
-            FrameUtils.flushFrame(outputFrame, outputWriter);
-        }
+        frameTupleAppender.flush(outputWriter, true);
     }
 
     public void close() throws HyracksDataException {
@@ -59,81 +54,42 @@ public class FrameTupleAppenderWrapper {
         outputWriter.fail();
     }
 
-    public void reset(ByteBuffer buffer, boolean clear) {
+    public void reset(IFrame buffer, boolean clear) throws HyracksDataException {
         frameTupleAppender.reset(buffer, clear);
     }
 
     public void appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length)
             throws HyracksDataException {
-        if (!frameTupleAppender.append(fieldSlots, bytes, offset, length)) {
-            FrameUtils.flushFrame(outputFrame, outputWriter);
-            frameTupleAppender.reset(outputFrame, true);
-            if (!frameTupleAppender.appendSkipEmptyField(fieldSlots, bytes, offset, length)) {
-                throw new HyracksDataException("The output cannot be fit into a frame.");
-            }
-        }
+        FrameUtils.appendSkipEmptyFieldToWriter(outputWriter, frameTupleAppender,
+                fieldSlots, bytes, offset, length);
     }
 
     public void append(byte[] bytes, int offset, int length) throws HyracksDataException {
-        if (!frameTupleAppender.append(bytes, offset, length)) {
-            FrameUtils.flushFrame(outputFrame, outputWriter);
-            frameTupleAppender.reset(outputFrame, true);
-            if (!frameTupleAppender.append(bytes, offset, length)) {
-                throw new HyracksDataException("The output cannot be fit into a frame.");
-            }
-        }
+        FrameUtils.appendToWriter(outputWriter, frameTupleAppender, bytes, offset, length);
     }
 
-    public void append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset) throws HyracksDataException {
-        if (!frameTupleAppender.append(tupleAccessor, tStartOffset, tEndOffset)) {
-            FrameUtils.flushFrame(outputFrame, outputWriter);
-            frameTupleAppender.reset(outputFrame, true);
-            if (!frameTupleAppender.append(tupleAccessor, tStartOffset, tEndOffset)) {
-                throw new HyracksDataException("The output cannot be fit into a frame.");
-            }
-        }
+    public void append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset)
+            throws HyracksDataException {
+        FrameUtils.appendToWriter(outputWriter, frameTupleAppender, tupleAccessor, tStartOffset, tEndOffset);
     }
 
     public void append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException {
-        if (!frameTupleAppender.append(tupleAccessor, tIndex)) {
-            FrameUtils.flushFrame(outputFrame, outputWriter);
-            frameTupleAppender.reset(outputFrame, true);
-            if (!frameTupleAppender.append(tupleAccessor, tIndex)) {
-                throw new HyracksDataException("The output cannot be fit into a frame.");
-            }
-        }
+        FrameUtils.appendToWriter(outputWriter, frameTupleAppender, tupleAccessor, tIndex);
     }
 
     public void appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
             throws HyracksDataException {
-        if (!frameTupleAppender.appendConcat(accessor0, tIndex0, accessor1, tIndex1)) {
-            FrameUtils.flushFrame(outputFrame, outputWriter);
-            frameTupleAppender.reset(outputFrame, true);
-            if (!frameTupleAppender.appendConcat(accessor0, tIndex0, accessor1, tIndex1)) {
-                throw new HyracksDataException("The output cannot be fit into a frame.");
-            }
-        }
+        FrameUtils.appendConcatToWriter(outputWriter, frameTupleAppender, accessor0, tIndex0, accessor1, tIndex1);
     }
 
     public void appendConcat(IFrameTupleAccessor accessor0, int tIndex0, int[] fieldSlots1, byte[] bytes1, int offset1,
             int dataLen1) throws HyracksDataException {
-        if (!frameTupleAppender.appendConcat(accessor0, tIndex0, fieldSlots1, bytes1, offset1, dataLen1)) {
-            FrameUtils.flushFrame(outputFrame, outputWriter);
-            frameTupleAppender.reset(outputFrame, true);
-            if (!frameTupleAppender.appendConcat(accessor0, tIndex0, fieldSlots1, bytes1, offset1, dataLen1)) {
-                throw new HyracksDataException("The output cannot be fit into a frame.");
-            }
-        }
+        FrameUtils.appendConcatToWriter(outputWriter, frameTupleAppender, accessor0, tIndex0,
+                fieldSlots1, bytes1, offset1, dataLen1);
     }
 
     public void appendProjection(IFrameTupleAccessor accessor, int tIndex, int[] fields) throws HyracksDataException {
-        if (!frameTupleAppender.appendProjection(accessor, tIndex, fields)) {
-            FrameUtils.flushFrame(outputFrame, outputWriter);
-            frameTupleAppender.reset(outputFrame, true);
-            if (!frameTupleAppender.appendProjection(accessor, tIndex, fields)) {
-                throw new HyracksDataException("The output cannot be fit into a frame.");
-            }
-        }
+        FrameUtils.appendProjectionToWriter(outputWriter, frameTupleAppender, accessor, tIndex, fields);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
index 471b1ef..0995564 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
@@ -15,79 +15,23 @@
 package edu.uci.ics.hyracks.dataflow.common.comm.io;
 
 import java.io.DataInputStream;
-import java.nio.ByteBuffer;
 
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
 import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
-public class ResultFrameTupleAccessor implements IFrameTupleAccessor {
+public class ResultFrameTupleAccessor extends FrameTupleAccessor {
 
-    private final int frameSize;
-    private ByteBuffer buffer;
-
-    public ResultFrameTupleAccessor(int frameSize) {
-        this.frameSize = frameSize;
-    }
-
-    @Override
-    public void reset(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    @Override
-    public ByteBuffer getBuffer() {
-        return buffer;
+    public ResultFrameTupleAccessor() {
+        super(null);
     }
 
     @Override
-    public int getTupleCount() {
-        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
-    }
-
-    @Override
-    public int getTupleStartOffset(int tupleIndex) {
-        return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * tupleIndex);
-    }
-
-    @Override
-    public int getTupleEndOffset(int tupleIndex) {
-        return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
-    }
-
-    @Override
-    public int getFieldStartOffset(int tupleIndex, int fIdx) {
-        return fIdx == 0 ? 0 : buffer.getInt(getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
-    }
-
-    @Override
-    public int getFieldEndOffset(int tupleIndex, int fIdx) {
-        return buffer.getInt(getTupleStartOffset(tupleIndex) + fIdx * 4);
-    }
-
-    @Override
-    public int getFieldLength(int tupleIndex, int fIdx) {
-        return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
-    }
-
-    @Override
-    public int getFieldSlotsLength() {
-        return getFieldCount() * 4;
-    }
-
-    public void prettyPrint() {
-        ByteBufferInputStream bbis = new ByteBufferInputStream();
-        DataInputStream dis = new DataInputStream(bbis);
-        int tc = getTupleCount();
-        System.err.println("TC: " + tc);
-        for (int i = 0; i < tc; ++i) {
-            System.err.print(i + ":(" + getTupleStartOffset(i) + ", " + getTupleEndOffset(i) + ")[");
+    protected void prettyPrint(int tid, ByteBufferInputStream bbis, DataInputStream dis, StringBuilder sb) {
+        sb.append(tid + ":(" + getTupleStartOffset(tid) + ", " + getTupleEndOffset(tid) + ")[");
 
-            bbis.setByteBuffer(buffer, getTupleStartOffset(i));
-            System.err.print(dis);
+        bbis.setByteBuffer(getBuffer(), getTupleStartOffset(tid));
+        sb.append(dis);
 
-            System.err.println("]");
-        }
+        sb.append("]\n");
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
index 4392fd6..3feeb58 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
@@ -14,21 +14,20 @@
  */
 package edu.uci.ics.hyracks.dataflow.common.comm.io;
 
-import java.nio.ByteBuffer;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
 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.util.FrameUtils;
 
 public class SerializingDataWriter implements IOpenableDataWriter<Object[]> {
     private static final Logger LOGGER = Logger.getLogger(SerializingDataWriter.class.getName());
 
-    private final ByteBuffer buffer;
-
     private final ArrayTupleBuilder tb;
 
     private final RecordDescriptor recordDescriptor;
@@ -41,20 +40,17 @@ public class SerializingDataWriter implements IOpenableDataWriter<Object[]> {
 
     public SerializingDataWriter(IHyracksTaskContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter)
             throws HyracksDataException {
-        buffer = ctx.allocateFrame();
         tb = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
         this.recordDescriptor = recordDescriptor;
         this.frameWriter = frameWriter;
-        tupleAppender = new FrameTupleAppender(ctx.getFrameSize());
+        tupleAppender = new FrameTupleAppender(new VSizeFrame(ctx));
         open = false;
     }
 
     @Override
     public void open() throws HyracksDataException {
         frameWriter.open();
-        buffer.clear();
         open = true;
-        tupleAppender.reset(buffer, true);
     }
 
     @Override
@@ -62,9 +58,7 @@ public class SerializingDataWriter implements IOpenableDataWriter<Object[]> {
         if (!open) {
             throw new HyracksDataException("Closing SerializingDataWriter that has not been opened");
         }
-        if (tupleAppender.getTupleCount() > 0) {
-            flushFrame();
-        }
+        tupleAppender.flush(frameWriter, true);
         frameWriter.close();
         open = false;
     }
@@ -82,22 +76,8 @@ public class SerializingDataWriter implements IOpenableDataWriter<Object[]> {
             }
             tb.addField(recordDescriptor.getFields()[i], instance);
         }
-        if (!tupleAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-            if (LOGGER.isLoggable(Level.FINEST)) {
-                LOGGER.finest("Flushing: position = " + buffer.position());
-            }
-            flushFrame();
-            tupleAppender.reset(buffer, true);
-            if (!tupleAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + buffer.capacity() + ")");
-            }
-        }
-    }
-
-    private void flushFrame() throws HyracksDataException {
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
-        frameWriter.nextFrame(buffer);
+        FrameUtils.appendToWriter(frameWriter, tupleAppender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+                tb.getSize());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/FrameUtils.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/FrameUtils.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/FrameUtils.java
index 4387e68..e2a9e5f 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/FrameUtils.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/FrameUtils.java
@@ -16,32 +16,268 @@ package edu.uci.ics.hyracks.dataflow.common.comm.util;
 
 import java.nio.ByteBuffer;
 
+import edu.uci.ics.hyracks.api.comm.IFrameFieldAppender;
 import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public class FrameUtils {
-    public static void copy(ByteBuffer srcFrame, ByteBuffer destFrame) {
-        makeReadable(srcFrame);
+
+    public static void copyWholeFrame(ByteBuffer srcFrame, ByteBuffer destFrame) {
+        srcFrame.clear();
         destFrame.clear();
         destFrame.put(srcFrame);
     }
 
-    public static void makeReadable(ByteBuffer frame) {
-        frame.position(0);
-        frame.limit(frame.capacity());
+    public static void copyAndFlip(ByteBuffer srcFrame, ByteBuffer destFrame) {
+        srcFrame.position(0);
+        destFrame.clear();
+        destFrame.put(srcFrame);
+        destFrame.flip();
     }
 
     public static void flushFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
         writer.nextFrame(buffer);
-        buffer.position(0);
-        buffer.limit(buffer.capacity());
+        buffer.clear();
+    }
+
+    /**
+     * A util function to append the data to appender. If the appender buffer is full, it will directly flush
+     * to the given writer, which saves the detecting logic in the caller.
+     * It will return the bytes that have been flushed.
+     *
+     * @param writer
+     * @param frameTupleAppender
+     * @param fieldSlots
+     * @param bytes
+     * @param offset
+     * @param length
+     * @return the number of bytes that have been flushed, 0 if not get flushed.
+     * @throws HyracksDataException
+     */
+    public static int appendSkipEmptyFieldToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+            int[] fieldSlots, byte[] bytes, int offset, int length) throws HyracksDataException {
+        int flushedBytes = 0;
+        if (!frameTupleAppender.appendSkipEmptyField(fieldSlots, bytes, offset, length)) {
+            flushedBytes = frameTupleAppender.getBuffer().capacity();
+            frameTupleAppender.flush(writer, true);
+            if (!frameTupleAppender.appendSkipEmptyField(fieldSlots, bytes, offset, length)) {
+                throw new HyracksDataException("The output cannot be fit into a frame.");
+            }
+        }
+        return flushedBytes;
+    }
+
+    /**
+     * A util function to append the data to appender. If the appender buffer is full, it will directly flush
+     * to the given writer, which saves the detecting logic in the caller.
+     * It will return the bytes that have been flushed.
+     *
+     * @param writer
+     * @param frameTupleAppender
+     * @param bytes
+     * @param offset
+     * @param length
+     * @return the number of bytes that have been flushed, 0 if not get flushed.
+     * @throws HyracksDataException
+     */
+    public static int appendToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender, byte[] bytes,
+            int offset, int length) throws HyracksDataException {
+        int flushedBytes = 0;
+        if (!frameTupleAppender.append(bytes, offset, length)) {
+            flushedBytes = frameTupleAppender.getBuffer().capacity();
+            frameTupleAppender.flush(writer, true);
+            if (!frameTupleAppender.append(bytes, offset, length)) {
+                throw new HyracksDataException("The output cannot be fit into a frame.");
+            }
+        }
+        return flushedBytes;
+    }
+
+    /**
+     * @param writer
+     * @param frameTupleAppender
+     * @param tupleAccessor
+     * @param tStartOffset
+     * @param tEndOffset
+     * @return the number of bytes that have been flushed, 0 if not get flushed.
+     * @throws HyracksDataException
+     */
+    public static int appendToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+            IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset)
+            throws HyracksDataException {
+        int flushedBytes = 0;
+        if (!frameTupleAppender.append(tupleAccessor, tStartOffset, tEndOffset)) {
+            flushedBytes = frameTupleAppender.getBuffer().capacity();
+            frameTupleAppender.flush(writer, true);
+            if (!frameTupleAppender.append(tupleAccessor, tStartOffset, tEndOffset)) {
+                throw new HyracksDataException("The output cannot be fit into a frame.");
+            }
+        }
+        return flushedBytes;
+    }
+
+    /**
+     * @param writer
+     * @param frameTupleAppender
+     * @param tupleAccessor
+     * @param tIndex
+     * @return the number of bytes that have been flushed, 0 if not get flushed.
+     * @throws HyracksDataException
+     */
+    public static int appendToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+            IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException {
+        int flushedBytes = 0;
+        if (!frameTupleAppender.append(tupleAccessor, tIndex)) {
+            flushedBytes = frameTupleAppender.getBuffer().capacity();
+            frameTupleAppender.flush(writer, true);
+            if (!frameTupleAppender.append(tupleAccessor, tIndex)) {
+                throw new HyracksDataException("The output cannot be fit into a frame.");
+            }
+        }
+        return flushedBytes;
+    }
+
+    /**
+     * @param writer
+     * @param tupleAppender
+     * @param fieldEndOffsets
+     * @param byteArray
+     * @param start
+     * @param size
+     * @return the number of bytes that have been flushed, 0 if not get flushed.
+     * @throws HyracksDataException
+     */
+    public static int appendToWriter(IFrameWriter writer, IFrameTupleAppender tupleAppender,
+            int[] fieldEndOffsets, byte[] byteArray, int start, int size) throws HyracksDataException {
+        int flushedBytes = 0;
+        if (!tupleAppender.append(fieldEndOffsets, byteArray, start, size)) {
+
+            flushedBytes = tupleAppender.getBuffer().capacity();
+            tupleAppender.flush(writer, true);
+
+            if (!tupleAppender.append(fieldEndOffsets, byteArray, start, size)) {
+                throw new HyracksDataException("The output cannot be fit into a frame.");
+            }
+        }
+        return flushedBytes;
+    }
+
+    /**
+     * @param writer
+     * @param frameTupleAppender
+     * @param accessor0
+     * @param tIndex0
+     * @param accessor1
+     * @param tIndex1
+     * @return the number of bytes that have been flushed, 0 if not get flushed.
+     * @throws HyracksDataException
+     */
+    public static int appendConcatToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+            IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
+            throws HyracksDataException {
+        int flushedBytes = 0;
+        if (!frameTupleAppender.appendConcat(accessor0, tIndex0, accessor1, tIndex1)) {
+            flushedBytes = frameTupleAppender.getBuffer().capacity();
+            frameTupleAppender.flush(writer, true);
+            if (!frameTupleAppender.appendConcat(accessor0, tIndex0, accessor1, tIndex1)) {
+                throw new HyracksDataException("The output cannot be fit into a frame.");
+            }
+        }
+        return flushedBytes;
+    }
+
+    /**
+     * @param writer
+     * @param frameTupleAppender
+     * @param accessor0
+     * @param tIndex0
+     * @param fieldSlots1
+     * @param bytes1
+     * @param offset1
+     * @param dataLen1
+     * @return the number of bytes that have been flushed, 0 if not get flushed.
+     * @throws HyracksDataException
+     */
+    public static int appendConcatToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+            IFrameTupleAccessor accessor0, int tIndex0, int[] fieldSlots1, byte[] bytes1, int offset1,
+            int dataLen1) throws HyracksDataException {
+        int flushedBytes = 0;
+        if (!frameTupleAppender.appendConcat(accessor0, tIndex0, fieldSlots1, bytes1, offset1, dataLen1)) {
+            flushedBytes = frameTupleAppender.getBuffer().capacity();
+            frameTupleAppender.flush(writer, true);
+            if (!frameTupleAppender.appendConcat(accessor0, tIndex0, fieldSlots1, bytes1, offset1, dataLen1)) {
+                throw new HyracksDataException("The output cannot be fit into a frame.");
+            }
+        }
+        return flushedBytes;
+    }
+
+    /**
+     * @param writer
+     * @param frameTupleAppender
+     * @param accessor
+     * @param tIndex
+     * @param fields
+     * @return the number of bytes that have been flushed, 0 if not get flushed.
+     * @throws HyracksDataException
+     */
+    public static int appendProjectionToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+            IFrameTupleAccessor accessor, int tIndex, int[] fields) throws HyracksDataException {
+        int flushedBytes = 0;
+        if (!frameTupleAppender.appendProjection(accessor, tIndex, fields)) {
+            flushedBytes = frameTupleAppender.getBuffer().capacity();
+            frameTupleAppender.flush(writer, true);
+            if (!frameTupleAppender.appendProjection(accessor, tIndex, fields)) {
+                throw new HyracksDataException("The output cannot be fit into a frame.");
+            }
+        }
+        return flushedBytes;
     }
 
-    public static int getAbsoluteFieldStartOffset(IFrameTupleAccessor accessor, int tuple, int field) {
-        return accessor.getTupleStartOffset(tuple) + accessor.getFieldSlotsLength()
-                + accessor.getFieldStartOffset(tuple, field);
+    /**
+     * @param writer
+     * @param appender
+     * @param array
+     * @param start
+     * @param length
+     * @return the number of bytes that have been flushed, 0 if not get flushed.
+     * @throws HyracksDataException
+     */
+    public static int appendFieldToWriter(IFrameWriter writer, IFrameFieldAppender appender, byte[] array,
+            int start, int length) throws HyracksDataException {
+        int flushedBytes = 0;
+        if (!appender.appendField(array, start, length)) {
+            flushedBytes = appender.getBuffer().capacity();
+            appender.flush(writer, true);
+            if (!appender.appendField(array, start, length)) {
+                throw new HyracksDataException("Could not write frame: the size of the tuple is too long");
+            }
+        }
+        return flushedBytes;
     }
+
+    /**
+     * @param writer
+     * @param appender
+     * @param accessor
+     * @param tid
+     * @param fid
+     * @return the number of bytes that have been flushed, 0 if not get flushed.
+     * @throws HyracksDataException
+     */
+    public static int appendFieldToWriter(IFrameWriter writer, IFrameFieldAppender appender,
+            IFrameTupleAccessor accessor, int tid, int fid) throws HyracksDataException {
+        int flushedBytes = 0;
+        if (!appender.appendField(accessor, tid, fid)) {
+            flushedBytes = appender.getBuffer().capacity();
+            appender.flush(writer, true);
+            if (!appender.appendField(accessor, tid, fid)) {
+                throw new HyracksDataException("Could not write frame: the size of the tuple is too long");
+            }
+        }
+        return flushedBytes;
+    }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java
index 31cd25c..be40d9f 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java
@@ -14,8 +14,8 @@
  */
 package edu.uci.ics.hyracks.dataflow.common.io;
 
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameReader;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
@@ -43,12 +43,30 @@ public class RunFileReader implements IFrameReader {
     }
 
     @Override
-    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        buffer.clear();
+    public boolean nextFrame(IFrame frame) throws HyracksDataException {
         if (readPtr >= size) {
             return false;
         }
-        readPtr += ioManager.syncRead(handle, readPtr, buffer);
+        frame.reset();
+        int readLength = ioManager.syncRead(handle, readPtr, frame.getBuffer());
+        if (readLength <= 0) {
+            throw new HyracksDataException("Premature end of file");
+        }
+        readPtr += readLength;
+        frame.ensureFrameSize(frame.getMinSize() * FrameHelper.deserializeNumOfMinFrame(frame.getBuffer()));
+        if (frame.getBuffer().hasRemaining()) {
+            if (readPtr < size) {
+                readLength = ioManager.syncRead(handle, readPtr, frame.getBuffer());
+                if (readLength < 0) {
+                    throw new HyracksDataException("Premature end of file");
+                }
+                readPtr += readLength;
+            }
+            if (frame.getBuffer().hasRemaining()) { // file is vanished.
+                FrameHelper.clearRemainingFrame(frame.getBuffer(), frame.getBuffer().position());
+            }
+        }
+        frame.getBuffer().flip();
         return true;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java
index 9faef09..4e2a985 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java
@@ -22,4 +22,10 @@ public class IntSerDeUtils {
                 + ((bytes[offset + 3] & 0xff) << 0);
     }
 
+    public static void putInt(byte[] bytes, int offset, int value) {
+        bytes[offset++] = (byte) (value >> 24);
+        bytes[offset++] = (byte) (value >> 16);
+        bytes[offset++] = (byte) (value >> 8);
+        bytes[offset++] = (byte) (value);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-common/src/test/java/edu/uci/ics/hyracks/dataflow/common/comm/io/largeobject/FrameFixedFieldTupleAppenderTest.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-common/src/test/java/edu/uci/ics/hyracks/dataflow/common/comm/io/largeobject/FrameFixedFieldTupleAppenderTest.java b/hyracks/hyracks-dataflow-common/src/test/java/edu/uci/ics/hyracks/dataflow/common/comm/io/largeobject/FrameFixedFieldTupleAppenderTest.java
new file mode 100644
index 0000000..ab9333c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/test/java/edu/uci/ics/hyracks/dataflow/common/comm/io/largeobject/FrameFixedFieldTupleAppenderTest.java
@@ -0,0 +1,215 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  you may obtain a copy of the License from
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.common.comm.io.largeobject;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameFixedFieldAppender;
+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.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+
+public class FrameFixedFieldTupleAppenderTest {
+
+    static final int INPUT_BUFFER_SIZE = 4096;
+    static final int TEST_FRAME_SIZE = 256;
+
+    FrameFixedFieldAppender appender;
+    static ISerializerDeserializer[] fields = new ISerializerDeserializer[] {
+            IntegerSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE,
+            IntegerSerializerDeserializer.INSTANCE,
+            UTF8StringSerializerDeserializer.INSTANCE,
+    };
+    static RecordDescriptor recordDescriptor = new RecordDescriptor(fields);
+    static ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
+
+    class SequetialDataVerifier implements IFrameWriter {
+
+        private final IFrameTupleAccessor accessor;
+        private IFrameTupleAccessor innerAccessor;
+        private int tid;
+
+        public SequetialDataVerifier(IFrameTupleAccessor accessor) {
+            this.accessor = accessor;
+            this.innerAccessor = new FrameTupleAccessor(recordDescriptor);
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            this.tid = 0;
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            innerAccessor.reset(buffer);
+            for (int i = 0; i < innerAccessor.getTupleCount(); ++i) {
+                validate(innerAccessor, i);
+            }
+        }
+
+        private void validate(IFrameTupleAccessor innerAccessor, int i) {
+            assertTrue(tid < accessor.getTupleCount());
+            assertEquals(accessor.getTupleLength(tid), innerAccessor.getTupleLength(i));
+            assertArrayEquals(Arrays.copyOfRange(accessor.getBuffer().array(), accessor.getTupleStartOffset(tid),
+                            accessor.getTupleEndOffset(tid)),
+                    Arrays.copyOfRange(innerAccessor.getBuffer().array(), innerAccessor.getTupleStartOffset(i),
+                            innerAccessor.getTupleEndOffset(i)));
+            tid++;
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+            assert false;
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            assertEquals(accessor.getTupleCount(), tid);
+        }
+    }
+
+    @Before
+    public void createAppender() throws HyracksDataException {
+        appender = new FrameFixedFieldAppender(fields.length);
+        FrameManager manager = new FrameManager(TEST_FRAME_SIZE);
+        IFrame frame = new VSizeFrame(manager);
+        appender.reset(frame, true);
+    }
+
+    private void testProcess(IFrameTupleAccessor accessor) throws HyracksDataException {
+        IFrameWriter writer = prepareValidator(accessor);
+        writer.open();
+        for (int tid = 0; tid < accessor.getTupleCount(); tid++) {
+            for (int fid = 0; fid < fields.length; fid++) {
+                if (!appender.appendField(accessor, tid, fid)) {
+                    appender.flush(writer, true);
+                    if (!appender.appendField(accessor, tid, fid)) {
+                    }
+                }
+            }
+        }
+        appender.flush(writer, true);
+        writer.close();
+    }
+
+    @Test
+    public void testAppendFieldShouldSucceed() throws HyracksDataException {
+        IFrameTupleAccessor accessor = prepareData(DATA_TYPE.NORMAL_RECORD);
+        testProcess(accessor);
+    }
+
+    @Test
+    public void testResetShouldWork() throws HyracksDataException {
+        testAppendFieldShouldSucceed();
+        appender.reset(new VSizeFrame(new FrameManager(TEST_FRAME_SIZE)), true);
+        testAppendFieldShouldSucceed();
+    }
+
+    private IFrameWriter prepareValidator(IFrameTupleAccessor accessor) throws HyracksDataException {
+        return new SequetialDataVerifier(accessor);
+    }
+
+    enum DATA_TYPE {
+        NORMAL_RECORD,
+        ONE_FIELD_LONG,
+        ONE_RECORD_LONG,
+    }
+
+    private IFrameTupleAccessor prepareData(DATA_TYPE type) throws HyracksDataException {
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(recordDescriptor);
+        IFrameTupleAppender appender = new FrameTupleAppender(
+                new VSizeFrame(new FrameManager(INPUT_BUFFER_SIZE)), true);
+        int i = 0;
+        do {
+            switch (type) {
+                case NORMAL_RECORD:
+                    makeATuple(tupleBuilder, i++);
+                    break;
+                case ONE_FIELD_LONG:
+                    makeASizeUpTuple(tupleBuilder, i++);
+                    break;
+                case ONE_RECORD_LONG:
+                    makeABigObjectTuple(tupleBuilder, i++);
+                    break;
+            }
+        } while (appender
+                .append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize()));
+        accessor.reset(appender.getBuffer());
+        return accessor;
+    }
+
+    private void makeATuple(ArrayTupleBuilder tupleBuilder, int i) throws HyracksDataException {
+        tupleBuilder.reset();
+        tupleBuilder.addField(fields[0], i);
+        tupleBuilder.addField(fields[1], String.valueOf(i));
+        tupleBuilder.addField(fields[2], -i);
+        tupleBuilder.addField(fields[3], String.valueOf(-i));
+    }
+
+    private String makeALongString(int length, char ch) {
+        char[] array = new char[length];
+        Arrays.fill(array, ch);
+        return new String(array);
+    }
+
+    private void makeASizeUpTuple(ArrayTupleBuilder tupleBuilder, int i) throws HyracksDataException {
+        tupleBuilder.reset();
+        tupleBuilder.addField(fields[0], i);
+        tupleBuilder.addField(fields[1], makeALongString(Math.min(Math.abs(1 << i), INPUT_BUFFER_SIZE), (char) i));
+        tupleBuilder.addField(fields[2], -i);
+        tupleBuilder.addField(fields[3], String.valueOf(-i));
+    }
+
+    private void makeABigObjectTuple(ArrayTupleBuilder tupleBuilder, int i) throws HyracksDataException {
+        tupleBuilder.reset();
+        tupleBuilder.addField(fields[0], i);
+        tupleBuilder.addField(fields[1], makeALongString(Math.min(i * 20, TEST_FRAME_SIZE), (char) i));
+        tupleBuilder.addField(fields[2], -i);
+        tupleBuilder.addField(fields[3], makeALongString(Math.min(i * 20, TEST_FRAME_SIZE), (char) i));
+    }
+
+    @Test
+    public void testAppendLargeFieldShouldSucceed() throws HyracksDataException {
+        IFrameTupleAccessor accessor = prepareData(DATA_TYPE.ONE_FIELD_LONG);
+        testProcess(accessor);
+    }
+
+    @Test
+    public void testAppendSmallFieldButLargeObjectWithShouldSucceed() throws HyracksDataException {
+        IFrameTupleAccessor accessor = prepareData(DATA_TYPE.ONE_RECORD_LONG);
+        testProcess(accessor);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
index 9bce999..0c06769 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
@@ -29,7 +29,6 @@
 package edu.uci.ics.hyracks.dataflow.hadoop;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -45,6 +44,7 @@ import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.ReflectionUtils;
 
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -205,9 +205,7 @@ public class HadoopReadOperatorDescriptor extends AbstractSingleActivityOperator
 
                     key = hadoopRecordReader.createKey();
                     value = hadoopRecordReader.createValue();
-                    ByteBuffer outBuffer = ctx.allocateFrame();
-                    FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-                    appender.reset(outBuffer, true);
+                    FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
                     RecordDescriptor outputRecordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
                             (Class<? extends Writable>) hadoopRecordReader.createKey().getClass(),
                             (Class<? extends Writable>) hadoopRecordReader.createValue().getClass());
@@ -223,18 +221,11 @@ public class HadoopReadOperatorDescriptor extends AbstractSingleActivityOperator
                                 case 1:
                                     tb.addField(outputRecordDescriptor.getFields()[1], value);
                             }
-                            if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                                FrameUtils.flushFrame(outBuffer, writer);
-                                appender.reset(outBuffer, true);
-                                if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                                    throw new HyracksDataException("Record size (" + tb.getSize()
-                                            + ") larger than frame size (" + outBuffer.capacity() + ")");
-                                }
-                            }
-                        }
-                        if (appender.getTupleCount() > 0) {
-                            FrameUtils.flushFrame(outBuffer, writer);
+                            FrameUtils
+                                    .appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(),
+                                            0, tb.getSize());
                         }
+                        appender.flush(writer, true);
                     } catch (Exception e) {
                         writer.fail();
                         throw new HyracksDataException(e);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/HadoopHelper.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/HadoopHelper.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/HadoopHelper.java
index 5ac55ff..95f53df 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/HadoopHelper.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/HadoopHelper.java
@@ -203,7 +203,7 @@ public class HadoopHelper {
 
     public int getSortFrameLimit(IHyracksCommonContext ctx) {
         int sortMemory = job.getConfiguration().getInt("io.sort.mb", 100);
-        return (int) (((long) sortMemory * 1024 * 1024) / ctx.getFrameSize());
+        return (int) (((long) sortMemory * 1024 * 1024) / ctx.getInitialFrameSize());
     }
 
     public Job getJob() {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/KVIterator.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/KVIterator.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/KVIterator.java
index 0bbb21f..070fc88 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/KVIterator.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/KVIterator.java
@@ -15,44 +15,42 @@
 package edu.uci.ics.hyracks.dataflow.hadoop.mapreduce;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.List;
 
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.mapred.RawKeyValueIterator;
 import org.apache.hadoop.util.Progress;
 
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
 
 public class KVIterator implements RawKeyValueIterator {
     private final HadoopHelper helper;
     private FrameTupleAccessor accessor;
     private DataInputBuffer kBuffer;
     private DataInputBuffer vBuffer;
-    private List<ByteBuffer> buffers;
+    private List<IFrame> buffers;
     private int bSize;
     private int bPtr;
     private int tIdx;
     private boolean eog;
 
-    public KVIterator(IHyracksTaskContext ctx, HadoopHelper helper, RecordDescriptor recordDescriptor) {
+    public KVIterator(HadoopHelper helper, RecordDescriptor recordDescriptor) {
         this.helper = helper;
-        accessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+        accessor = new FrameTupleAccessor(recordDescriptor);
         kBuffer = new DataInputBuffer();
         vBuffer = new DataInputBuffer();
     }
 
-    void reset(List<ByteBuffer> buffers, int bSize) {
+    void reset(List<IFrame> buffers, int bSize) {
         this.buffers = buffers;
         this.bSize = bSize;
         bPtr = 0;
         tIdx = 0;
         eog = false;
         if (bSize > 0) {
-            accessor.reset(buffers.get(0));
+            accessor.reset(buffers.get(0).getBuffer());
             tIdx = -1;
         } else {
             eog = true;
@@ -83,14 +81,14 @@ public class KVIterator implements RawKeyValueIterator {
                     continue;
                 }
                 tIdx = -1;
-                accessor.reset(buffers.get(bPtr));
+                accessor.reset(buffers.get(bPtr).getBuffer());
                 continue;
             }
             kBuffer.reset(accessor.getBuffer().array(),
-                    FrameUtils.getAbsoluteFieldStartOffset(accessor, tIdx, helper.KEY_FIELD_INDEX),
+                    accessor.getAbsoluteFieldStartOffset(tIdx, helper.KEY_FIELD_INDEX),
                     accessor.getFieldLength(tIdx, helper.KEY_FIELD_INDEX));
             vBuffer.reset(accessor.getBuffer().array(),
-                    FrameUtils.getAbsoluteFieldStartOffset(accessor, tIdx, helper.VALUE_FIELD_INDEX),
+                    accessor.getAbsoluteFieldStartOffset(tIdx, helper.KEY_FIELD_INDEX),
                     accessor.getFieldLength(tIdx, helper.VALUE_FIELD_INDEX));
             break;
         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb-hyracks/blob/0d87a57f/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
index 2bef21a..bf6e2cf 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
@@ -29,7 +29,9 @@ import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 
+import edu.uci.ics.hyracks.api.comm.IFrame;
 import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
@@ -40,13 +42,12 @@ import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
 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;
+import edu.uci.ics.hyracks.dataflow.hadoop.util.MRContextUtil;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
 import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
-import edu.uci.ics.hyracks.dataflow.hadoop.util.MRContextUtil;
 
 public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable, K2 extends Writable, V2 extends Writable>
         extends AbstractSingleActivityOperatorDescriptor {
@@ -82,16 +83,15 @@ public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable,
 
         class SortingRecordWriter extends RecordWriter<K2, V2> {
             private final ArrayTupleBuilder tb;
-            private final ByteBuffer frame;
+            private final IFrame frame;
             private final FrameTupleAppender fta;
             private ExternalSortRunGenerator runGen;
             private int blockId;
 
             public SortingRecordWriter() throws HyracksDataException {
                 tb = new ArrayTupleBuilder(2);
-                frame = ctx.allocateFrame();
-                fta = new FrameTupleAppender(ctx.getFrameSize());
-                fta.reset(frame, true);
+                frame = new VSizeFrame(ctx);
+                fta = new FrameTupleAppender(frame);
             }
 
             public void initBlock(int blockId) throws HyracksDataException {
@@ -113,31 +113,29 @@ public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable,
                 value.write(dos);
                 tb.addFieldEndOffset();
                 if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                    runGen.nextFrame(frame);
+                    runGen.nextFrame(frame.getBuffer());
                     fta.reset(frame, true);
                     if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                         throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
-                                + frame.capacity() + ")");
+                                + frame.getBuffer().capacity() + ")");
                     }
                 }
             }
 
             public void sortAndFlushBlock(final IFrameWriter writer) throws HyracksDataException {
                 if (fta.getTupleCount() > 0) {
-                    runGen.nextFrame(frame);
+                    runGen.nextFrame(frame.getBuffer());
                     fta.reset(frame, true);
                 }
                 runGen.close();
                 IFrameWriter delegatingWriter = new IFrameWriter() {
-                    private final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
-                    private final ByteBuffer outFrame = ctx.allocateFrame();
-                    private final FrameTupleAccessor fta = new FrameTupleAccessor(ctx.getFrameSize(),
+                    private final FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
+                    private final FrameTupleAccessor fta = new FrameTupleAccessor(
                             helper.getMapOutputRecordDescriptorWithoutExtraFields());
                     private final ArrayTupleBuilder tb = new ArrayTupleBuilder(3);
 
                     @Override
                     public void open() throws HyracksDataException {
-                        appender.reset(outFrame, true);
                     }
 
                     @Override
@@ -155,8 +153,7 @@ public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable,
                             }
                             tb.addFieldEndOffset();
                             if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                                FrameUtils.flushFrame(outFrame, writer);
-                                appender.reset(outFrame, true);
+                                appender.flush(writer, true);
                                 if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                                     throw new IllegalStateException();
                                 }
@@ -166,9 +163,7 @@ public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable,
 
                     @Override
                     public void close() throws HyracksDataException {
-                        if (appender.getTupleCount() > 0) {
-                            FrameUtils.flushFrame(outFrame, writer);
-                        }
+                        appender.flush(writer, true);
                     }
 
                     @Override
@@ -183,12 +178,10 @@ public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable,
                     TaskAttemptContext ctaskAttemptContext = helper.createTaskAttemptContext(taId);
                     final IFrameWriter outputWriter = delegatingWriter;
                     RecordWriter<K2, V2> recordWriter = new RecordWriter<K2, V2>() {
-                        private final FrameTupleAppender fta = new FrameTupleAppender(ctx.getFrameSize());
-                        private final ByteBuffer buffer = ctx.allocateFrame();
+                        private final FrameTupleAppender fta = new FrameTupleAppender(new VSizeFrame(ctx));
                         private final ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
 
                         {
-                            fta.reset(buffer, true);
                             outputWriter.open();
                         }
 
@@ -201,8 +194,7 @@ public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable,
                             value.write(dos);
                             tb.addFieldEndOffset();
                             if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-                                FrameUtils.flushFrame(buffer, outputWriter);
-                                fta.reset(buffer, true);
+                                fta.flush(outputWriter, true);
                                 if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
                                     throw new IllegalStateException();
                                 }
@@ -211,10 +203,7 @@ public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable,
 
                         @Override
                         public void close(TaskAttemptContext context) throws IOException, InterruptedException {
-                            if (fta.getTupleCount() > 0) {
-                                FrameUtils.flushFrame(buffer, outputWriter);
-                                outputWriter.close();
-                            }
+                            fta.flush(outputWriter, true);
                         }
                     };
                     delegatingWriter = new ReduceWriter<K2, V2, K2, V2>(ctx, helper,
@@ -226,7 +215,7 @@ public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable,
                 for (int i = 0; i < comparatorFactories.length; ++i) {
                     comparators[i] = comparatorFactories[i].createBinaryComparator();
                 }
-                ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, runGen.getFrameSorter(),
+                ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, runGen.getSorter(),
                         runGen.getRuns(), new int[] { 0 }, comparators, null,
                         helper.getMapOutputRecordDescriptorWithoutExtraFields(), framesLimit, delegatingWriter);
                 merger.process();
@@ -253,8 +242,9 @@ public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable,
                                 Thread.currentThread().setContextClassLoader(ctxCL);
                             }
                             recordWriter.initBlock(blockId);
-                            Mapper<K1, V1, K2, V2>.Context mCtx = new MRContextUtil().createMapContext(conf, taId, recordReader,
-                                    recordWriter, null, null, split);
+                            Mapper<K1, V1, K2, V2>.Context mCtx = new MRContextUtil()
+                                    .createMapContext(conf, taId, recordReader,
+                                            recordWriter, null, null, split);
                             mapper.run(mCtx);
                             recordReader.close();
                             recordWriter.sortAndFlushBlock(writer);