Copied hyracks trunk into fullstack

git-svn-id: https://hyracks.googlecode.com/svn/branches/fullstack_staging@1958 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks/hyracks-dataflow-std/pom.xml b/hyracks/hyracks-dataflow-std/pom.xml
new file mode 100644
index 0000000..550814d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/pom.xml
@@ -0,0 +1,49 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>edu.uci.ics.hyracks</groupId>
+  <artifactId>hyracks-dataflow-std</artifactId>
+  <version>0.2.2-SNAPSHOT</version>
+
+  <parent>
+    <groupId>edu.uci.ics.hyracks</groupId>
+    <artifactId>hyracks</artifactId>
+    <version>0.2.2-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>2.0.2</version>
+        <configuration>
+          <source>1.6</source>
+          <target>1.6</target>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  <dependencies>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-api</artifactId>
+  		<version>0.2.2-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>edu.uci.ics.hyracks</groupId>
+  		<artifactId>hyracks-dataflow-common</artifactId>
+  		<version>0.2.2-SNAPSHOT</version>
+  		<type>jar</type>
+  		<scope>compile</scope>
+  	</dependency>
+  	<dependency>
+  		<groupId>junit</groupId>
+  		<artifactId>junit</artifactId>
+  		<version>4.8.2</version>
+  		<type>jar</type>
+  		<scope>test</scope>
+  	</dependency>
+  </dependencies>
+</project>
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java
new file mode 100644
index 0000000..11a02b3
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+
+public abstract class AbstractActivityNode implements IActivity {
+    private static final long serialVersionUID = 1L;
+
+    protected final ActivityId id;
+
+    public AbstractActivityNode(ActivityId id) {
+        this.id = id;
+    }
+
+    @Override
+    public ActivityId getActivityId() {
+        return id;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
new file mode 100644
index 0000000..ef479e1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.job.ActivityCluster;
+import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
+
+public abstract class AbstractConnectorDescriptor implements IConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+    protected final ConnectorDescriptorId id;
+
+    protected String displayName;
+
+    public AbstractConnectorDescriptor(IConnectorDescriptorRegistry spec) {
+        this.id = spec.createConnectorDescriptor(this);
+        displayName = getClass().getName() + "[" + id + "]";
+    }
+
+    public ConnectorDescriptorId getConnectorId() {
+        return id;
+    }
+
+    public String getDisplayName() {
+        return displayName;
+    }
+
+    public void setDisplayName(String displayName) {
+        this.displayName = displayName;
+    }
+
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject jconn = new JSONObject();
+
+        jconn.put("id", String.valueOf(getConnectorId()));
+        jconn.put("java-class", getClass().getName());
+        jconn.put("display-name", displayName);
+
+        return jconn;
+    }
+
+    @Override
+    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ActivityCluster ac,
+            ICCApplicationContext appCtx) {
+        // do nothing
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
new file mode 100644
index 0000000..8e9f131
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
+
+public abstract class AbstractMToNConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public AbstractMToNConnectorDescriptor(IConnectorDescriptorRegistry spec) {
+        super(spec);
+    }
+
+    @Override
+    public void indicateTargetPartitions(int nProducerPartitions, int nConsumerPartitions, int producerIndex,
+            BitSet targetBitmap) {
+        targetBitmap.clear();
+        targetBitmap.set(0, nConsumerPartitions);
+    }
+
+    @Override
+    public void indicateSourcePartitions(int nProducerPartitions, int nConsumerPartitions, int consumerIndex,
+            BitSet sourceBitmap) {
+        sourceBitmap.clear();
+        sourceBitmap.set(0, nProducerPartitions);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
new file mode 100644
index 0000000..153abd3
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+
+public abstract class AbstractOperatorDescriptor implements IOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    protected final OperatorDescriptorId odId;
+
+    protected String[] partitions;
+
+    protected final RecordDescriptor[] recordDescriptors;
+
+    protected final int inputArity;
+
+    protected final int outputArity;
+
+    protected String displayName;
+
+    public AbstractOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity, int outputArity) {
+        odId = spec.createOperatorDescriptorId(this);
+        this.inputArity = inputArity;
+        this.outputArity = outputArity;
+        recordDescriptors = new RecordDescriptor[outputArity];
+        displayName = getClass().getName() + "[" + odId + "]";
+    }
+
+    @Override
+    public final OperatorDescriptorId getOperatorId() {
+        return odId;
+    }
+
+    @Override
+    public int getInputArity() {
+        return inputArity;
+    }
+
+    @Override
+    public int getOutputArity() {
+        return outputArity;
+    }
+
+    @Override
+    public RecordDescriptor[] getOutputRecordDescriptors() {
+        return recordDescriptors;
+    }
+
+    public String getDisplayName() {
+        return displayName;
+    }
+
+    public void setDisplayName(String displayName) {
+        this.displayName = displayName;
+    }
+
+    @Override
+    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ICCApplicationContext appCtx) {
+        // do nothing
+    }
+
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject jop = new JSONObject();
+        jop.put("id", String.valueOf(getOperatorId()));
+        jop.put("java-class", getClass().getName());
+        jop.put("in-arity", getInputArity());
+        jop.put("out-arity", getOutputArity());
+        jop.put("display-name", displayName);
+        return jop;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java
new file mode 100644
index 0000000..d8e90a3
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorNodePushable.java
@@ -0,0 +1,10 @@
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+
+public abstract class AbstractOperatorNodePushable implements IOperatorNodePushable {
+    @Override
+    public String getDisplayName() {
+        return toString();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
new file mode 100644
index 0000000..9048670
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+
+public abstract class AbstractSingleActivityOperatorDescriptor extends AbstractOperatorDescriptor implements IActivity {
+    private static final long serialVersionUID = 1L;
+
+    protected final ActivityId activityNodeId;
+
+    public AbstractSingleActivityOperatorDescriptor(IOperatorDescriptorRegistry spec, int inputArity, int outputArity) {
+        super(spec, inputArity, outputArity);
+        activityNodeId = new ActivityId(odId, 0);
+    }
+
+    @Override
+    public ActivityId getActivityId() {
+        return activityNodeId;
+    }
+
+    @Override
+    public final void contributeActivities(IActivityGraphBuilder builder) {
+        builder.addActivity(this, this);
+        for (int i = 0; i < getInputArity(); ++i) {
+            builder.addSourceEdge(i, this, i);
+        }
+        for (int i = 0; i < getOutputArity(); ++i) {
+            builder.addTargetEdge(i, this, i);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractStateObject.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractStateObject.java
new file mode 100644
index 0000000..753fb98
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractStateObject.java
@@ -0,0 +1,47 @@
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public abstract class AbstractStateObject implements IStateObject {
+    protected JobId jobId;
+
+    protected Object id;
+
+    protected long memoryOccupancy;
+
+    protected AbstractStateObject() {
+    }
+
+    protected AbstractStateObject(JobId jobId, Object id) {
+        this.jobId = jobId;
+        this.id = id;
+    }
+
+    @Override
+    public final JobId getJobId() {
+        return jobId;
+    }
+
+    public final void setJobId(JobId jobId) {
+        this.jobId = jobId;
+    }
+
+    @Override
+    public final Object getId() {
+        return id;
+    }
+
+    public final void setId(Object id) {
+        this.id = id;
+    }
+
+    @Override
+    public final long getMemoryOccupancy() {
+        return memoryOccupancy;
+    }
+
+    public void setMemoryOccupancy(long memoryOccupancy) {
+        this.memoryOccupancy = memoryOccupancy;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryInputOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryInputOperatorNodePushable.java
new file mode 100644
index 0000000..c6ef746
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryInputOperatorNodePushable.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractUnaryInputOperatorNodePushable extends AbstractOperatorNodePushable implements
+        IFrameWriter {
+    protected IFrameWriter writer;
+    protected RecordDescriptor recordDesc;
+
+    @Override
+    public final IFrameWriter getInputFrameWriter(int index) {
+        return this;
+    }
+
+    @Override
+    public final int getInputArity() {
+        return 1;
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+    }
+
+    @Override
+    public void deinitialize() throws HyracksDataException {
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryInputSinkOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryInputSinkOperatorNodePushable.java
new file mode 100644
index 0000000..12e8cb4
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryInputSinkOperatorNodePushable.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public abstract class AbstractUnaryInputSinkOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
+    @Override
+    public final void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+        throw new IllegalStateException();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryInputUnaryOutputOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryInputUnaryOutputOperatorNodePushable.java
new file mode 100644
index 0000000..e3767b3
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryInputUnaryOutputOperatorNodePushable.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+public abstract class AbstractUnaryInputUnaryOutputOperatorNodePushable extends AbstractUnaryOutputOperatorNodePushable
+        implements IFrameWriter {
+    @Override
+    public final IFrameWriter getInputFrameWriter(int index) {
+        return this;
+    }
+
+    @Override
+    public final int getInputArity() {
+        return 1;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryOutputOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryOutputOperatorNodePushable.java
new file mode 100644
index 0000000..1ce7281
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryOutputOperatorNodePushable.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class AbstractUnaryOutputOperatorNodePushable extends AbstractOperatorNodePushable {
+    protected IFrameWriter writer;
+    protected RecordDescriptor recordDesc;
+
+    @Override
+    public final void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+        if (index != 0) {
+            throw new IllegalStateException();
+        }
+        this.writer = writer;
+        this.recordDesc = recordDesc;
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+    }
+
+    @Override
+    public void deinitialize() throws HyracksDataException {
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryOutputSourceOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryOutputSourceOperatorNodePushable.java
new file mode 100644
index 0000000..db09e57
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractUnaryOutputSourceOperatorNodePushable.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+public abstract class AbstractUnaryOutputSourceOperatorNodePushable extends AbstractUnaryOutputOperatorNodePushable {
+    @Override
+    public IFrameWriter getInputFrameWriter(int index) {
+        throw new IllegalStateException();
+    }
+
+    @Override
+    public final int getInputArity() {
+        return 0;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/IOpenableDataWriterOperator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/IOpenableDataWriterOperator.java
new file mode 100644
index 0000000..b802c6f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/IOpenableDataWriterOperator.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.base;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+
+public interface IOpenableDataWriterOperator extends IOpenableDataWriter<Object[]> {
+    public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer);
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/AbstractPartitionCollector.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/AbstractPartitionCollector.java
new file mode 100644
index 0000000..b2dad8f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/AbstractPartitionCollector.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public abstract class AbstractPartitionCollector implements IPartitionCollector {
+    protected final IHyracksTaskContext ctx;
+
+    protected final ConnectorDescriptorId connectorId;
+
+    protected final int receiverIndex;
+
+    public AbstractPartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId, int receiverIndex) {
+        this.ctx = ctx;
+        this.connectorId = connectorId;
+        this.receiverIndex = receiverIndex;
+    }
+
+    @Override
+    public JobId getJobId() {
+        return ctx.getJobletContext().getJobId();
+    }
+
+    @Override
+    public ConnectorDescriptorId getConnectorId() {
+        return connectorId;
+    }
+
+    @Override
+    public int getReceiverIndex() {
+        return receiverIndex;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/IPartitionAcceptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/IPartitionAcceptor.java
new file mode 100644
index 0000000..8942ea9
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/IPartitionAcceptor.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public interface IPartitionAcceptor {
+    public void addPartition(PartitionId pid, IInputChannel channel);
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/IPartitionBatchManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/IPartitionBatchManager.java
new file mode 100644
index 0000000..4962d64
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/IPartitionBatchManager.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IPartitionBatchManager extends IPartitionAcceptor {
+    public void getNextBatch(List<IFrameReader> batch, int size) throws HyracksDataException;
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..713fb0e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
@@ -0,0 +1,77 @@
+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.channels.IInputChannelMonitor;
+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;
+
+public class InputChannelFrameReader implements IFrameReader, IInputChannelMonitor {
+    private final IInputChannel channel;
+
+    private int availableFrames;
+
+    private boolean eos;
+
+    private boolean failed;
+
+    public InputChannelFrameReader(IInputChannel channel) {
+        this.channel = channel;
+        availableFrames = 0;
+        eos = false;
+        failed = false;
+    }
+
+    @Override
+    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");
+            }
+            if (availableFrames <= 0 && eos) {
+                return false;
+            }
+            --availableFrames;
+        }
+        ByteBuffer srcBuffer = channel.getNextBuffer();
+        FrameUtils.copy(srcBuffer, buffer);
+        channel.recycleBuffer(srcBuffer);
+        return true;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+
+    }
+
+    @Override
+    public synchronized void notifyFailure(IInputChannel channel) {
+        failed = true;
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+        availableFrames += nFrames;
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void notifyEndOfStream(IInputChannel channel) {
+        eos = true;
+        notifyAll();
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..578bfec
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
@@ -0,0 +1,163 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import java.util.BitSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class NonDeterministicChannelReader implements IInputChannelMonitor, IPartitionAcceptor {
+    private static final Logger LOGGER = Logger.getLogger(NonDeterministicChannelReader.class.getName());
+
+    private final int nSenderPartitions;
+
+    private final IInputChannel[] channels;
+
+    private final BitSet frameAvailability;
+
+    private final int[] availableFrameCounts;
+
+    private final BitSet eosSenders;
+
+    private final BitSet failSenders;
+
+    private final BitSet closedSenders;
+
+    private int lastReadSender;
+
+    public NonDeterministicChannelReader(int nSenderPartitions, BitSet expectedPartitions) {
+        this.nSenderPartitions = nSenderPartitions;
+        channels = new IInputChannel[nSenderPartitions];
+        eosSenders = new BitSet(nSenderPartitions);
+        failSenders = new BitSet(nSenderPartitions);
+        closedSenders = new BitSet(nSenderPartitions);
+        closedSenders.or(expectedPartitions);
+        closedSenders.flip(0, nSenderPartitions);
+        frameAvailability = new BitSet(nSenderPartitions);
+        availableFrameCounts = new int[nSenderPartitions];
+    }
+
+    @Override
+    public void addPartition(PartitionId pid, IInputChannel channel) {
+        channel.registerMonitor(this);
+        channel.setAttachment(pid);
+        synchronized (this) {
+            channels[pid.getSenderIndex()] = channel;
+        }
+    }
+
+    public int getSenderPartitionCount() {
+        return nSenderPartitions;
+    }
+
+    public void open() throws HyracksDataException {
+        lastReadSender = 0;
+    }
+
+    public IInputChannel[] getChannels() {
+        return channels;
+    }
+
+    public synchronized int findNextSender() throws HyracksDataException {
+        while (true) {
+            switch (lastReadSender) {
+                default:
+                    lastReadSender = frameAvailability.nextSetBit(lastReadSender + 1);
+                    if (lastReadSender >= 0) {
+                        break;
+                    }
+                case 0:
+                    lastReadSender = frameAvailability.nextSetBit(0);
+            }
+            if (lastReadSender >= 0) {
+                assert availableFrameCounts[lastReadSender] > 0;
+                if (--availableFrameCounts[lastReadSender] == 0) {
+                    frameAvailability.clear(lastReadSender);
+                }
+                return lastReadSender;
+            }
+            if (!failSenders.isEmpty()) {
+                throw new HyracksDataException("Failure occurred on input");
+            }
+            for (int i = eosSenders.nextSetBit(0); i >= 0; i = eosSenders.nextSetBit(i)) {
+                channels[i].close();
+                eosSenders.clear(i);
+                closedSenders.set(i);
+            }
+            int nextClosedBitIndex = closedSenders.nextClearBit(0);
+            if (nextClosedBitIndex < 0 || nextClosedBitIndex >= nSenderPartitions) {
+                lastReadSender = -1;
+                return lastReadSender;
+            }
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    public synchronized void close() throws HyracksDataException {
+        for (int i = closedSenders.nextClearBit(0); i >= 0 && i < nSenderPartitions; i = closedSenders
+                .nextClearBit(i + 1)) {
+            if (channels[i] != null) {
+                channels[i].close();
+                channels[i] = null;
+            }
+        }
+    }
+
+    @Override
+    public synchronized void notifyFailure(IInputChannel channel) {
+        PartitionId pid = (PartitionId) channel.getAttachment();
+        int senderIndex = pid.getSenderIndex();
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Failure: " + pid.getConnectorDescriptorId() + " sender: " + senderIndex + " receiver: "
+                    + pid.getReceiverIndex());
+        }
+        failSenders.set(senderIndex);
+        eosSenders.set(senderIndex);
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+        PartitionId pid = (PartitionId) channel.getAttachment();
+        int senderIndex = pid.getSenderIndex();
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Data available: " + pid.getConnectorDescriptorId() + " sender: " + senderIndex + " receiver: "
+                    + pid.getReceiverIndex());
+        }
+        availableFrameCounts[senderIndex] += nFrames;
+        frameAvailability.set(senderIndex);
+        notifyAll();
+    }
+
+    @Override
+    public synchronized void notifyEndOfStream(IInputChannel channel) {
+        PartitionId pid = (PartitionId) channel.getAttachment();
+        int senderIndex = pid.getSenderIndex();
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("EOS: " + pid);
+        }
+        eosSenders.set(senderIndex);
+        notifyAll();
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..657165c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicFrameReader.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+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;
+
+public class NonDeterministicFrameReader implements IFrameReader {
+    private final NonDeterministicChannelReader channelReader;
+
+    public NonDeterministicFrameReader(NonDeterministicChannelReader channelReader) {
+        this.channelReader = channelReader;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        channelReader.open();
+    }
+
+    @Override
+    public boolean nextFrame(ByteBuffer buffer) 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;
+        }
+        return false;
+    }
+
+    @Override
+    public synchronized void close() throws HyracksDataException {
+        channelReader.close();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionBatchManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionBatchManager.java
new file mode 100644
index 0000000..ba25850
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionBatchManager.java
@@ -0,0 +1,78 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class NonDeterministicPartitionBatchManager implements IPartitionBatchManager {
+    private final IInputChannel[] channels;
+
+    private List<IFrameReader> partitions;
+
+    private List<IFrameReader> batch;
+
+    private int requiredSize;
+
+    public NonDeterministicPartitionBatchManager(int nSenders) {
+        channels = new IInputChannel[nSenders];
+        partitions = new ArrayList<IFrameReader>();
+    }
+
+    @Override
+    public synchronized void addPartition(PartitionId pid, IInputChannel channel) {
+        channels[pid.getSenderIndex()] = channel;
+        InputChannelFrameReader channelReader = new InputChannelFrameReader(channel);
+        channel.registerMonitor(channelReader);
+        if (batch != null && batch.size() < requiredSize) {
+            batch.add(channelReader);
+            if (batch.size() == requiredSize) {
+                notifyAll();
+            }
+        } else {
+            partitions.add(channelReader);
+        }
+    }
+
+    @Override
+    public synchronized void getNextBatch(List<IFrameReader> batch, int size) throws HyracksDataException {
+        if (partitions.size() <= size) {
+            batch.addAll(partitions);
+            partitions.clear();
+        } else if (partitions.size() > size) {
+            List<IFrameReader> sublist = partitions.subList(0, size);
+            batch.addAll(sublist);
+            sublist.clear();
+        }
+        if (batch.size() == size) {
+            return;
+        }
+        this.batch = batch;
+        this.requiredSize = size;
+        while (batch.size() < size) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        this.batch = null;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/PartitionCollector.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/PartitionCollector.java
new file mode 100644
index 0000000..23df4bb
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/PartitionCollector.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class PartitionCollector extends AbstractPartitionCollector {
+    private final BitSet expectedPartitions;
+
+    private final IFrameReader frameReader;
+
+    private final IPartitionAcceptor pa;
+
+    public PartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId, int receiverIndex,
+            BitSet expectedPartitions, IFrameReader frameReader, IPartitionAcceptor pa) {
+        super(ctx, connectorId, receiverIndex);
+        this.expectedPartitions = expectedPartitions;
+        this.frameReader = frameReader;
+        this.pa = pa;
+    }
+
+    @Override
+    public void open() throws HyracksException {
+    }
+
+    @Override
+    public void addPartitions(Collection<PartitionChannel> partitions) throws HyracksException {
+        for (PartitionChannel pc : partitions) {
+            PartitionId pid = pc.getPartitionId();
+            IInputChannel channel = pc.getInputChannel();
+            pa.addPartition(pid, channel);
+            channel.open(ctx);
+        }
+    }
+
+    @Override
+    public IFrameReader getReader() throws HyracksException {
+        return frameReader;
+    }
+
+    @Override
+    public void close() throws HyracksException {
+
+    }
+
+    @Override
+    public Collection<PartitionId> getRequiredPartitionIds() throws HyracksException {
+        Collection<PartitionId> c = new ArrayList<PartitionId>(expectedPartitions.cardinality());
+        for (int i = expectedPartitions.nextSetBit(0); i >= 0; i = expectedPartitions.nextSetBit(i + 1)) {
+            c.add(new PartitionId(getJobId(), getConnectorId(), i, getReceiverIndex()));
+        }
+        return c;
+    }
+
+    @Override
+    public void abort() {
+
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..5f41069
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.collectors;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+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.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
+
+public class SortMergeFrameReader implements IFrameReader {
+    private IHyracksTaskContext ctx;
+    private final int maxConcurrentMerges;
+    private final int nSenders;
+    private final int[] sortFields;
+    private final IBinaryComparator[] comparators;
+    private final RecordDescriptor recordDescriptor;
+    private final IPartitionBatchManager pbm;
+
+    private RunMergingFrameReader merger;
+
+    public SortMergeFrameReader(IHyracksTaskContext ctx, int maxConcurrentMerges, int nSenders, int[] sortFields,
+            IBinaryComparator[] comparators, RecordDescriptor recordDescriptor, IPartitionBatchManager pbm) {
+        this.ctx = ctx;
+        this.maxConcurrentMerges = maxConcurrentMerges;
+        this.nSenders = nSenders;
+        this.sortFields = sortFields;
+        this.comparators = comparators;
+        this.recordDescriptor = recordDescriptor;
+        this.pbm = pbm;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        if (maxConcurrentMerges >= nSenders) {
+            List<ByteBuffer> inFrames = new ArrayList<ByteBuffer>();
+            for (int i = 0; i < nSenders; ++i) {
+                inFrames.add(ByteBuffer.allocate(ctx.getFrameSize()));
+            }
+            List<IFrameReader> batch = new ArrayList<IFrameReader>();
+            pbm.getNextBatch(batch, nSenders);
+            merger = new RunMergingFrameReader(ctx, batch.toArray(new IFrameReader[nSenders]), inFrames, sortFields,
+                    comparators, recordDescriptor);
+        } else {
+            // multi level merge.
+            throw new HyracksDataException("Not yet supported");
+        }
+        merger.open();
+    }
+
+    @Override
+    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        buffer.position(buffer.capacity());
+        buffer.limit(buffer.capacity());
+        return merger.nextFrame(buffer);
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        merger.close();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/GlobalHashingLocalityMap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/GlobalHashingLocalityMap.java
new file mode 100644
index 0000000..f3086f0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/GlobalHashingLocalityMap.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.connectors;
+
+public class GlobalHashingLocalityMap implements ILocalityMap {
+
+    private static final long serialVersionUID = 1L;
+
+    private int[] consumers;
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.examples.text.client.aggregation.helpers.ILocalityMap#getConsumers(int)
+     */
+    @Override
+    public int[] getConsumers(int senderID, int nConsumerPartitions) {
+        if (consumers == null) {
+            consumers = new int[nConsumerPartitions];
+            for (int i = 0; i < consumers.length; i++) {
+                consumers[i] = i;
+            }
+        }
+        return consumers;
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.examples.text.client.aggregation.helpers.ILocalityMap#getConsumerPartitionCount()
+     */
+    @Override
+    public int getConsumerPartitionCount(int nConsumerPartitions) {
+        return nConsumerPartitions;
+    }
+
+    @Override
+    public boolean isConnected(int senderID, int receiverID, int nConsumerPartitions) {
+        return true;
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashtableLocalityMap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashtableLocalityMap.java
new file mode 100644
index 0000000..a187a59
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashtableLocalityMap.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+public class HashtableLocalityMap implements ILocalityMap {
+
+    private static final long serialVersionUID = 1L;
+
+    private final BitSet nodeMap;
+
+    public HashtableLocalityMap(BitSet nodeMap) {
+        this.nodeMap = nodeMap;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.examples.text.client.aggregation.helpers.ILocalityMap
+     * #getConsumers(int, int)
+     */
+    @Override
+    public int[] getConsumers(int senderID, int nConsumerPartitions) {
+        int consumersForSender = 0;
+        // Get the count of consumers
+        for (int i = senderID * nConsumerPartitions; i < (senderID + 1) * nConsumerPartitions; i++) {
+            if (nodeMap.get(i))
+                consumersForSender++;
+        }
+        int[] cons = new int[consumersForSender];
+        int consIdx = 0;
+        for (int i = senderID * nConsumerPartitions; i < (senderID + 1) * nConsumerPartitions; i++) {
+            if (nodeMap.get(i)) {
+                cons[consIdx] = i - senderID * nConsumerPartitions;
+                consIdx++;
+            }
+        }
+        return cons;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.examples.text.client.aggregation.helpers.ILocalityMap
+     * #getConsumerPartitionCount(int)
+     */
+    @Override
+    public int getConsumerPartitionCount(int nConsumerPartitions) {
+        return nConsumerPartitions;
+    }
+
+    @Override
+    public boolean isConnected(int senderID, int receiverID, int nConsumerPartitions) {
+        return nodeMap.get(senderID * nConsumerPartitions + receiverID);
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/ILocalityMap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/ILocalityMap.java
new file mode 100644
index 0000000..b30971d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/ILocalityMap.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.connectors;
+
+import java.io.Serializable;
+
+public interface ILocalityMap extends Serializable {
+
+    public int[] getConsumers(int senderID, int nConsumerPartitions);
+
+    public boolean isConnected(int senderID, int receiverID, int nConsumerPartitions);
+
+    public int getConsumerPartitionCount(int nConsumerPartitions);
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
new file mode 100644
index 0000000..0349633
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.collectors.PartitionCollector;
+
+public class LocalityAwareMToNPartitioningConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private ILocalityMap localityMap;
+
+    private ITuplePartitionComputerFactory tpcf;
+
+    public LocalityAwareMToNPartitioningConnectorDescriptor(IConnectorDescriptorRegistry spec, ITuplePartitionComputerFactory tpcf,
+            ILocalityMap localityMap) {
+        super(spec);
+        this.localityMap = localityMap;
+        this.tpcf = tpcf;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor#createPartitioner
+     * (edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory, int, int, int)
+     */
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        return new LocalityAwarePartitionDataWriter(ctx, edwFactory, recordDesc, tpcf.createPartitioner(),
+                nConsumerPartitions, localityMap, index);
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor#
+     * createPartitionCollector
+     * (edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int, int, int)
+     */
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int receiverIndex, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        BitSet expectedPartitions = new BitSet(nProducerPartitions);
+        for (int i = 0; i < nProducerPartitions; i++) {
+            if (localityMap.isConnected(i, receiverIndex, nConsumerPartitions))
+                expectedPartitions.set(i);
+        }
+        NonDeterministicChannelReader channelReader = new NonDeterministicChannelReader(nProducerPartitions,
+                expectedPartitions);
+        NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
+        return new PartitionCollector(ctx, getConnectorId(), receiverIndex, expectedPartitions, frameReader,
+                channelReader);
+    }
+
+}
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
new file mode 100644
index 0000000..6ec9013
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
@@ -0,0 +1,125 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.connectors;
+
+import java.io.IOException;
+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.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;
+
+public class LocalityAwarePartitionDataWriter implements IFrameWriter {
+
+    private final IFrameWriter[] pWriters;
+    private final FrameTupleAppender[] appenders;
+    private final FrameTupleAccessor tupleAccessor;
+    private final ITuplePartitionComputer tpc;
+
+    public LocalityAwarePartitionDataWriter(IHyracksTaskContext ctx, IPartitionWriterFactory pwFactory,
+            RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc, int nConsumerPartitions,
+            ILocalityMap localityMap, int senderIndex) throws HyracksDataException {
+        int[] consumerPartitions = localityMap.getConsumers(senderIndex, nConsumerPartitions);
+        pWriters = new IFrameWriter[consumerPartitions.length];
+        appenders = new FrameTupleAppender[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);
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+        this.tpc = tpc;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.api.comm.IFrameWriter#open()
+     */
+    @Override
+    public void open() throws HyracksDataException {
+        for (int i = 0; i < pWriters.length; ++i) {
+            pWriters[i].open();
+            appenders[i].reset(appenders[i].getBuffer(), true);
+        }
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.api.comm.IFrameWriter#nextFrame(java.nio.ByteBuffer)
+     */
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        tupleAccessor.reset(buffer);
+        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 IllegalStateException();
+                }
+            }
+        }
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.api.comm.IFrameWriter#fail()
+     */
+    @Override
+    public void fail() throws HyracksDataException {
+        for (int i = 0; i < appenders.length; ++i) {
+            pWriters[i].fail();
+        }
+    }
+
+    private void flushFrame(ByteBuffer buffer, IFrameWriter frameWriter) throws HyracksDataException {
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+        frameWriter.nextFrame(buffer);
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.api.comm.IFrameWriter#close()
+     */
+    @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]);
+            }
+            pWriters[i].close();
+        }
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
new file mode 100644
index 0000000..451e208
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.collectors.PartitionCollector;
+
+public class MToNPartitioningConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private ITuplePartitionComputerFactory tpcf;
+
+    public MToNPartitioningConnectorDescriptor(IConnectorDescriptorRegistry spec, ITuplePartitionComputerFactory tpcf) {
+        super(spec);
+        this.tpcf = tpcf;
+    }
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory,
+                recordDesc, tpcf.createPartitioner());
+        return hashWriter;
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        BitSet expectedPartitions = new BitSet(nProducerPartitions);
+        expectedPartitions.set(0, nProducerPartitions);
+        NonDeterministicChannelReader channelReader = new NonDeterministicChannelReader(nProducerPartitions,
+                expectedPartitions);
+        NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, frameReader, channelReader);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
new file mode 100644
index 0000000..0d80161
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+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.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.IPartitionBatchManager;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicPartitionBatchManager;
+import edu.uci.ics.hyracks.dataflow.std.collectors.PartitionCollector;
+import edu.uci.ics.hyracks.dataflow.std.collectors.SortMergeFrameReader;
+
+public class MToNPartitioningMergingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final ITuplePartitionComputerFactory tpcf;
+    private final int[] sortFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final boolean stable;
+
+    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec, ITuplePartitionComputerFactory tpcf,
+            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories) {
+        this(spec, tpcf, sortFields, comparatorFactories, false);
+    }
+
+    public MToNPartitioningMergingConnectorDescriptor(IConnectorDescriptorRegistry spec, ITuplePartitionComputerFactory tpcf,
+            int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, boolean stable) {
+        super(spec);
+        this.tpcf = tpcf;
+        this.sortFields = sortFields;
+        this.comparatorFactories = comparatorFactories;
+        this.stable = stable;
+    }
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory,
+                recordDesc, tpcf.createPartitioner());
+        return hashWriter;
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        IPartitionBatchManager pbm = new NonDeterministicPartitionBatchManager(nProducerPartitions);
+        IFrameReader sortMergeFrameReader = new SortMergeFrameReader(ctx, nProducerPartitions, nProducerPartitions,
+                sortFields, comparators, recordDesc, pbm);
+        BitSet expectedPartitions = new BitSet();
+        expectedPartitions.set(0, nProducerPartitions);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sortMergeFrameReader, pbm);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
new file mode 100644
index 0000000..2d04938
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.connectors;
+
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.collectors.PartitionCollector;
+
+public class MToNReplicatingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+    public MToNReplicatingConnectorDescriptor(IConnectorDescriptorRegistry spec) {
+        super(spec);
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
+        for (int i = 0; i < nConsumerPartitions; ++i) {
+            epWriters[i] = edwFactory.createFrameWriter(i);
+        }
+        return new IFrameWriter() {
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                buffer.mark();
+                for (int i = 0; i < epWriters.length; ++i) {
+                    if (i != 0) {
+                        buffer.reset();
+                    }
+                    epWriters[i].nextFrame(buffer);
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                for (int i = 0; i < epWriters.length; ++i) {
+                    epWriters[i].fail();
+                }
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                for (int i = 0; i < epWriters.length; ++i) {
+                    epWriters[i].close();
+                }
+            }
+
+            @Override
+            public void open() throws HyracksDataException {
+                for (int i = 0; i < epWriters.length; ++i) {
+                    epWriters[i].open();
+                }
+            }
+        };
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        BitSet expectedPartitions = new BitSet(nProducerPartitions);
+        expectedPartitions.set(0, nProducerPartitions);
+        NonDeterministicChannelReader channelReader = new NonDeterministicChannelReader(nProducerPartitions,
+                expectedPartitions);
+        NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, frameReader, channelReader);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
new file mode 100644
index 0000000..7d007a4
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.ActivityCluster;
+import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.collectors.PartitionCollector;
+
+public class OneToOneConnectorDescriptor extends AbstractConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public OneToOneConnectorDescriptor(IConnectorDescriptorRegistry spec) {
+        super(spec);
+    }
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        return edwFactory.createFrameWriter(index);
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        BitSet expectedPartitions = new BitSet(nProducerPartitions);
+        expectedPartitions.set(index);
+        NonDeterministicChannelReader channelReader = new NonDeterministicChannelReader(nProducerPartitions,
+                expectedPartitions);
+        NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, frameReader, channelReader);
+    }
+
+    @Override
+    public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, ActivityCluster ac,
+            ICCApplicationContext appCtx) {
+        OperatorDescriptorId consumer = ac.getConsumerActivity(getConnectorId()).getOperatorDescriptorId();
+        OperatorDescriptorId producer = ac.getProducerActivity(getConnectorId()).getOperatorDescriptorId();
+
+        constraintAcceptor.addConstraint(new Constraint(new PartitionCountExpression(consumer),
+                new PartitionCountExpression(producer)));
+    }
+
+    @Override
+    public void indicateTargetPartitions(int nProducerPartitions, int nConsumerPartitions, int producerIndex,
+            BitSet targetBitmap) {
+        targetBitmap.clear();
+        targetBitmap.set(producerIndex);
+    }
+
+    @Override
+    public void indicateSourcePartitions(int nProducerPartitions, int nConsumerPartitions, int consumerIndex,
+            BitSet sourceBitmap) {
+        sourceBitmap.clear();
+        sourceBitmap.set(consumerIndex);
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..6b3f3c0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.connectors;
+
+import java.io.IOException;
+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.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;
+
+public class PartitionDataWriter implements IFrameWriter {
+    private final int consumerPartitionCount;
+    private final IFrameWriter[] pWriters;
+    private final FrameTupleAppender[] appenders;
+    private final FrameTupleAccessor tupleAccessor;
+    private final ITuplePartitionComputer tpc;
+
+    public PartitionDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount, IPartitionWriterFactory pwFactory,
+            RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc) throws HyracksDataException {
+        this.consumerPartitionCount = consumerPartitionCount;
+        pWriters = new IFrameWriter[consumerPartitionCount];
+        appenders = new FrameTupleAppender[consumerPartitionCount];
+        for (int i = 0; i < consumerPartitionCount; ++i) {
+            try {
+                pWriters[i] = pwFactory.createFrameWriter(i);
+                appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
+                appenders[i].reset(ctx.allocateFrame(), true);
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+        this.tpc = tpc;
+    }
+
+    @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]);
+            }
+            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) {
+            pWriters[i].open();
+            appenders[i].reset(appenders[i].getBuffer(), true);
+        }
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        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 IllegalStateException();
+                }
+            }
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        for (int i = 0; i < appenders.length; ++i) {
+            pWriters[i].fail();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..1cd898e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public abstract class AbstractDeserializedFileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    protected FileSplit[] splits;
+
+    public AbstractDeserializedFileScanOperatorDescriptor(IOperatorDescriptorRegistry spec, FileSplit[] splits,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 0, 1);
+        recordDescriptors[0] = recordDescriptor;
+        this.splits = splits;
+    }
+
+    protected abstract IRecordReader createRecordReader(File file, RecordDescriptor desc) throws Exception;
+
+    protected abstract void configure() throws Exception;
+
+    protected class DeserializedFileScanOperator implements IOpenableDataWriterOperator {
+        private IOpenableDataWriter<Object[]> writer;
+        private int index;
+
+        DeserializedFileScanOperator(int index) {
+            this.index = index;
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            if (index != 0) {
+                throw new IndexOutOfBoundsException("Invalid index: " + index);
+            }
+            this.writer = writer;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            FileSplit split = splits[index];
+            RecordDescriptor desc = recordDescriptors[0];
+            IRecordReader reader;
+            try {
+                reader = createRecordReader(split.getLocalFile().getFile(), desc);
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+            if (desc == null) {
+                desc = recordDescriptors[0];
+            }
+            writer.open();
+            try {
+                while (true) {
+                    Object[] record = new Object[desc.getFieldCount()];
+                    if (!reader.read(record)) {
+                        break;
+                    }
+                    writer.writeData(record);
+                }
+            } catch (Exception e) {
+                writer.fail();
+                throw new HyracksDataException(e);
+            } finally {
+                reader.close();
+                writer.close();
+            }
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            // do nothing
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+            // do nothing
+        }
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new DeserializedOperatorNodePushable(ctx, new DeserializedFileScanOperator(partition), null);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
new file mode 100644
index 0000000..e05d80d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public abstract class AbstractFileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    protected class FileWriteOperator implements IOpenableDataWriterOperator {
+        private int index;
+        private IRecordWriter writer;
+
+        FileWriteOperator(int index) {
+            this.index = index;
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            throw new IllegalArgumentException();
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            FileSplit split = splits[index];
+            try {
+                writer = createRecordWriter(split, index);
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            writer.close();
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            try {
+                writer.write(data);
+
+            } catch (Exception e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    protected FileSplit[] splits;
+
+    public FileSplit[] getSplits() {
+        return splits;
+    }
+
+    public void setSplits(FileSplit[] splits) {
+        this.splits = splits;
+    }
+
+    public AbstractFileWriteOperatorDescriptor(IOperatorDescriptorRegistry spec, FileSplit[] splits) {
+        super(spec, 1, 0);
+        this.splits = splits;
+    }
+
+    protected abstract IRecordWriter createRecordWriter(FileSplit fileSplit, int index) throws Exception;
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new DeserializedOperatorNodePushable(ctx, new FileWriteOperator(partition),
+                recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ConstantFileSplitProvider.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ConstantFileSplitProvider.java
new file mode 100644
index 0000000..21f5e3f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ConstantFileSplitProvider.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+public class ConstantFileSplitProvider implements IFileSplitProvider {
+    private static final long serialVersionUID = 1L;
+    private final FileSplit[] splits;
+
+    public ConstantFileSplitProvider(FileSplit[] splits) {
+        this.splits = splits;
+    }
+
+    @Override
+    public FileSplit[] getFileSplits() {
+        return splits;
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..7cf437d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
@@ -0,0 +1,252 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+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.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.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParser;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+
+public class DelimitedDataTupleParserFactory implements ITupleParserFactory {
+    private static final long serialVersionUID = 1L;
+    private IValueParserFactory[] valueParserFactories;
+    private char fieldDelimiter;
+
+    public DelimitedDataTupleParserFactory(IValueParserFactory[] fieldParserFactories, char fieldDelimiter) {
+        this.valueParserFactories = fieldParserFactories;
+        this.fieldDelimiter = fieldDelimiter;
+    }
+
+    @Override
+    public ITupleParser createTupleParser(final IHyracksTaskContext ctx) {
+        return new ITupleParser() {
+            @Override
+            public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
+                try {
+                    IValueParser[] valueParsers = new IValueParser[valueParserFactories.length];
+                    for (int i = 0; i < valueParserFactories.length; ++i) {
+                        valueParsers[i] = valueParserFactories[i].createValueParser();
+                    }
+                    ByteBuffer frame = ctx.allocateFrame();
+                    FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+                    appender.reset(frame, true);
+                    ArrayTupleBuilder tb = new ArrayTupleBuilder(valueParsers.length);
+                    DataOutput dos = tb.getDataOutput();
+
+                    FieldCursor cursor = new FieldCursor(new InputStreamReader(in));
+                    while (cursor.nextRecord()) {
+                        tb.reset();
+                        for (int i = 0; i < valueParsers.length; ++i) {
+                            if (!cursor.nextField()) {
+                                break;
+                            }
+                            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 IllegalStateException();
+                            }
+                        }
+                    }
+                    if (appender.getTupleCount() > 0) {
+                        FrameUtils.flushFrame(frame, writer);
+                    }
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        };
+    }
+
+    private enum State {
+        INIT, IN_RECORD, EOR, CR, EOF
+    }
+
+    private class FieldCursor {
+        private static final int INITIAL_BUFFER_SIZE = 4096;
+        private static final int INCREMENT = 4096;
+
+        private final Reader in;
+
+        private char[] buffer;
+        private int start;
+        private int end;
+        private State state;
+
+        private int fStart;
+        private int fEnd;
+
+        public FieldCursor(Reader in) {
+            this.in = in;
+            buffer = new char[INITIAL_BUFFER_SIZE];
+            start = 0;
+            end = 0;
+            state = State.INIT;
+        }
+
+        public boolean nextRecord() throws IOException {
+            while (true) {
+                switch (state) {
+                    case INIT:
+                        boolean eof = !readMore();
+                        if (eof) {
+                            state = State.EOF;
+                            return false;
+                        } else {
+                            state = State.IN_RECORD;
+                            return true;
+                        }
+
+                    case IN_RECORD:
+                        int p = start;
+                        while (true) {
+                            if (p >= end) {
+                                int s = start;
+                                eof = !readMore();
+                                if (eof) {
+                                    state = State.EOF;
+                                    return start < end;
+                                }
+                                p -= (s - start);
+                            }
+                            char ch = buffer[p];
+                            if (ch == '\n') {
+                                start = p + 1;
+                                state = State.EOR;
+                                break;
+                            } else if (ch == '\r') {
+                                start = p + 1;
+                                state = State.CR;
+                                break;
+                            }
+                            ++p;
+                        }
+                        break;
+
+                    case CR:
+                        if (start >= end) {
+                            eof = !readMore();
+                            if (eof) {
+                                state = State.EOF;
+                                return false;
+                            }
+                        }
+                        char ch = buffer[start];
+                        if (ch == '\n') {
+                            ++start;
+                            state = State.EOR;
+                        } else {
+                            state = State.IN_RECORD;
+                            return true;
+                        }
+
+                    case EOR:
+                        if (start >= end) {
+                            eof = !readMore();
+                            if (eof) {
+                                state = State.EOF;
+                                return false;
+                            }
+                        }
+                        state = State.IN_RECORD;
+                        return start < end;
+
+                    case EOF:
+                        return false;
+                }
+            }
+        }
+
+        public boolean nextField() throws IOException {
+            switch (state) {
+                case INIT:
+                case EOR:
+                case EOF:
+                case CR:
+                    return false;
+
+                case IN_RECORD:
+                    boolean eof;
+                    int p = start;
+                    while (true) {
+                        if (p >= end) {
+                            int s = start;
+                            eof = !readMore();
+                            if (eof) {
+                                state = State.EOF;
+                                return true;
+                            }
+                            p -= (s - start);
+                        }
+                        char ch = buffer[p];
+                        if (ch == fieldDelimiter) {
+                            fStart = start;
+                            fEnd = p;
+                            start = p + 1;
+                            return true;
+                        } else if (ch == '\n') {
+                            fStart = start;
+                            fEnd = p;
+                            start = p + 1;
+                            state = State.EOR;
+                            return true;
+                        } else if (ch == '\r') {
+                            fStart = start;
+                            fEnd = p;
+                            start = p + 1;
+                            state = State.CR;
+                            return true;
+                        }
+                        ++p;
+                    }
+            }
+            throw new IllegalStateException();
+        }
+
+        private boolean readMore() throws IOException {
+            if (start > 0) {
+                System.arraycopy(buffer, start, buffer, 0, end - start);
+            }
+            end -= start;
+            start = 0;
+
+            if (end == buffer.length) {
+                buffer = Arrays.copyOf(buffer, buffer.length + INCREMENT);
+            }
+
+            int n = in.read(buffer, end, buffer.length - end);
+            if (n < 0) {
+                return false;
+            }
+            end += n;
+            return true;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
new file mode 100644
index 0000000..d78593b
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+
+public class FileScanOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final IFileSplitProvider fileSplitProvider;
+
+    private final ITupleParserFactory tupleParserFactory;
+
+    public FileScanOperatorDescriptor(IOperatorDescriptorRegistry spec, IFileSplitProvider fileSplitProvider,
+            ITupleParserFactory tupleParserFactory, RecordDescriptor rDesc) {
+        super(spec, 0, 1);
+        this.fileSplitProvider = fileSplitProvider;
+        this.tupleParserFactory = tupleParserFactory;
+        recordDescriptors[0] = rDesc;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        final FileSplit split = fileSplitProvider.getFileSplits()[partition];
+        final ITupleParser tp = tupleParserFactory.createTupleParser(ctx);
+        return new AbstractUnaryOutputSourceOperatorNodePushable() {
+            @Override
+            public void initialize() throws HyracksDataException {
+                File f = split.getLocalFile().getFile();
+                writer.open();
+                try {
+                    InputStream in;
+                    try {
+                        in = new FileInputStream(f);
+                    } catch (FileNotFoundException e) {
+                        writer.fail();
+                        throw new HyracksDataException(e);
+                    }
+                    tp.parse(in, writer);
+                } catch (Exception e) {
+                    writer.fail();
+                    throw new HyracksDataException(e);
+                } finally {
+                    writer.close();
+                }
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
new file mode 100644
index 0000000..b74d97b
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileSplit.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.File;
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.io.FileReference;
+
+public class FileSplit implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final String nodeName;
+
+    private final FileReference file;
+
+    public FileSplit(String nodeName, FileReference file) {
+        this.nodeName = nodeName;
+        this.file = file;
+    }
+
+    public FileSplit(String nodeName, String path) {
+        this.nodeName = nodeName;
+        this.file = new FileReference(new File(path));
+    }
+
+    public String getNodeName() {
+        return nodeName;
+    }
+
+    public FileReference getLocalFile() {
+        return file;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
new file mode 100644
index 0000000..7f257ea
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+
+public class FrameFileWriterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private IFileSplitProvider fileSplitProvider;
+
+    public FrameFileWriterOperatorDescriptor(IOperatorDescriptorRegistry spec, IFileSplitProvider fileSplitProvider) {
+        super(spec, 1, 0);
+        this.fileSplitProvider = fileSplitProvider;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+        final FileSplit[] splits = fileSplitProvider.getFileSplits();
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
+            private OutputStream out;
+
+            @Override
+            public void open() throws HyracksDataException {
+                try {
+                    out = new FileOutputStream(splits[partition].getLocalFile().getFile());
+                } catch (FileNotFoundException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                try {
+                    out.write(buffer.array());
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                try {
+                    out.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IFileSplitProvider.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IFileSplitProvider.java
new file mode 100644
index 0000000..90f65ab
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IFileSplitProvider.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.Serializable;
+
+public interface IFileSplitProvider extends Serializable {
+    public FileSplit[] getFileSplits();
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordReader.java
new file mode 100644
index 0000000..5d94c12
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordReader.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+public interface IRecordReader {
+
+    public boolean read(Object[] record) throws Exception;
+
+    public void close();
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordWriter.java
new file mode 100644
index 0000000..679088f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/IRecordWriter.java
@@ -0,0 +1,25 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.File;
+
+public interface IRecordWriter {
+
+    public void close();
+
+    public void write(Object[] record) throws Exception;
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParser.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParser.java
new file mode 100644
index 0000000..5014e41
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParser.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.InputStream;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ITupleParser {
+    public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
new file mode 100644
index 0000000..eb051f5
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public interface ITupleParserFactory extends Serializable {
+    public ITupleParser createTupleParser(IHyracksTaskContext ctx);
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileWriteOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileWriteOperatorDescriptor.java
new file mode 100644
index 0000000..d5256ee
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/LineFileWriteOperatorDescriptor.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.OutputStream;
+
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+
+public class LineFileWriteOperatorDescriptor extends AbstractFileWriteOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private static class LineWriterImpl extends RecordWriter {
+        LineWriterImpl(File file, int[] columns, char separator) throws Exception {
+            super(columns, separator, new Object[] { file });
+        }
+
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public OutputStream createOutputStream(Object[] args) throws Exception {
+            return new FileOutputStream((File) args[0]);
+        }
+    }
+
+    private int[] columns;
+    private char separator;
+
+    public LineFileWriteOperatorDescriptor(IOperatorDescriptorRegistry spec, FileSplit[] splits) {
+        this(spec, splits, null, RecordWriter.COMMA);
+    }
+
+    public LineFileWriteOperatorDescriptor(IOperatorDescriptorRegistry spec, FileSplit[] splits, int[] columns) {
+        this(spec, splits, columns, RecordWriter.COMMA);
+    }
+
+    public LineFileWriteOperatorDescriptor(IOperatorDescriptorRegistry spec, FileSplit[] splits, int[] columns, char separator) {
+        super(spec, splits);
+        this.columns = columns;
+        this.separator = separator;
+    }
+
+    @Override
+    protected IRecordWriter createRecordWriter(FileSplit fileSplit, int index) throws Exception {
+        return new LineWriterImpl(fileSplit.getLocalFile().getFile(), columns, separator);
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..6fb0189
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.BufferedWriter;
+import java.io.DataInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.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.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+
+/**
+ * File writer to output plain text.
+ */
+public class PlainFileWriterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    /**
+     * 
+     */
+    private static final long serialVersionUID = 1L;
+
+    private IFileSplitProvider fileSplitProvider;
+
+    private String delim;
+
+    /**
+     * @param spec
+     * @param inputArity
+     * @param outputArity
+     */
+    public PlainFileWriterOperatorDescriptor(IOperatorDescriptorRegistry spec, IFileSplitProvider fileSplitProvider,
+            String delim) {
+        super(spec, 1, 0);
+        this.fileSplitProvider = fileSplitProvider;
+        this.delim = delim;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.api.dataflow.IActivityNode#createPushRuntime(edu.
+     * uci.ics.hyracks.api.context.IHyracksContext,
+     * edu.uci.ics.hyracks.api.job.IOperatorEnvironment,
+     * edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider, int,
+     * int)
+     */
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+            throws HyracksDataException {
+        // Output files
+        final FileSplit[] splits = fileSplitProvider.getFileSplits();
+        // Frame accessor
+        final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
+                recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
+        // Record descriptor
+        final RecordDescriptor recordDescriptor = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
+            private BufferedWriter out;
+
+            private ByteBufferInputStream bbis;
+
+            private DataInputStream di;
+
+            @Override
+            public void open() throws HyracksDataException {
+                try {
+                    out = new BufferedWriter(new FileWriter(splits[partition].getLocalFile().getFile()));
+                    bbis = new ByteBufferInputStream();
+                    di = new DataInputStream(bbis);
+                } catch (Exception e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                try {
+                    frameTupleAccessor.reset(buffer);
+                    for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+                        int start = frameTupleAccessor.getTupleStartOffset(tIndex)
+                                + frameTupleAccessor.getFieldSlotsLength();
+                        bbis.setByteBuffer(buffer, start);
+                        Object[] record = new Object[recordDescriptor.getFieldCount()];
+                        for (int i = 0; i < record.length; ++i) {
+                            Object instance = recordDescriptor.getFields()[i].deserialize(di);
+                            if (i == 0) {
+                                out.write(String.valueOf(instance));
+                            } else {
+                                out.write(delim + String.valueOf(instance));
+                            }
+                        }
+                        out.write("\n");
+                    }
+                } catch (IOException ex) {
+                    throw new HyracksDataException(ex);
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                try {
+                    out.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordFileScanOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordFileScanOperatorDescriptor.java
new file mode 100644
index 0000000..36e07c4
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordFileScanOperatorDescriptor.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+
+public class RecordFileScanOperatorDescriptor extends AbstractDeserializedFileScanOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public RecordFileScanOperatorDescriptor(IOperatorDescriptorRegistry spec, FileSplit[] splits, RecordDescriptor recordDescriptor) {
+        super(spec, splits, recordDescriptor);
+    }
+
+    private static class RecordReaderImpl implements IRecordReader {
+        private RecordDescriptor recordDesc;
+        private DataInputStream in;
+
+        RecordReaderImpl(File file, RecordDescriptor recordDesc) throws Exception {
+            this.in = new DataInputStream(new BufferedInputStream(new FileInputStream(file)));
+            this.recordDesc = recordDesc;
+        }
+
+        @Override
+        public boolean read(Object[] record) throws Exception {
+            in.mark(1);
+            if (in.read() < 0) {
+                return false;
+            }
+            in.reset();
+            for (int i = 0; i < record.length; ++i) {
+                record[i] = recordDesc.getFields()[i].deserialize(in);
+            }
+            return true;
+        }
+
+        @Override
+        public void close() {
+            try {
+                in.close();
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    @Override
+    protected IRecordReader createRecordReader(File file, RecordDescriptor desc) throws Exception {
+        return new RecordReaderImpl(file, desc);
+    }
+
+    @Override
+    protected void configure() throws Exception {
+        // currently a no-op, but is meant to initialize , if required before it
+        // is asked
+        // to create a record reader
+        // this is executed at the node and is useful for operators that could
+        // not be
+        // initialized from the client completely, because of lack of
+        // information specific
+        // to the node where the operator gets executed.
+
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordWriter.java
new file mode 100644
index 0000000..1858a73
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/RecordWriter.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.file;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+
+import edu.uci.ics.hyracks.dataflow.std.util.StringSerializationUtils;
+
+public abstract class RecordWriter implements IRecordWriter {
+
+    protected final BufferedWriter bufferedWriter;
+    protected final int[] columns;
+    protected final char separator;
+
+    public static final char COMMA = ',';
+
+    public RecordWriter(Object[] args) throws Exception {
+        OutputStream outputStream = createOutputStream(args);
+        if (outputStream != null) {
+            bufferedWriter = new BufferedWriter(new OutputStreamWriter(createOutputStream(args)));
+        } else {
+            bufferedWriter = null;
+        }
+        this.columns = null;
+        this.separator = COMMA;
+    }
+
+    public RecordWriter(int[] columns, char separator, Object[] args) throws Exception {
+        OutputStream outputStream = createOutputStream(args);
+        if (outputStream != null) {
+            bufferedWriter = new BufferedWriter(new OutputStreamWriter(outputStream));
+        } else {
+            bufferedWriter = null;
+        }
+        this.columns = columns;
+        this.separator = separator;
+    }
+
+    @Override
+    public void close() {
+        try {
+            bufferedWriter.close();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @Override
+    public void write(Object[] record) throws Exception {
+        if (columns == null) {
+            for (int i = 0; i < record.length; ++i) {
+                if (i != 0) {
+                    bufferedWriter.write(separator);
+                }
+                bufferedWriter.write(StringSerializationUtils.toString(record[i]));
+            }
+        } else {
+            for (int i = 0; i < columns.length; ++i) {
+                if (i != 0) {
+                    bufferedWriter.write(separator);
+                }
+                bufferedWriter.write(StringSerializationUtils.toString(record[columns[i]]));
+            }
+        }
+        bufferedWriter.write("\n");
+    }
+
+    public abstract OutputStream createOutputStream(Object[] args) throws Exception;
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AggregateState.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AggregateState.java
new file mode 100644
index 0000000..e72f85c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/AggregateState.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.io.Serializable;
+
+/**
+ *
+ */
+public class AggregateState implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    public Object state = null;
+
+    public AggregateState() {
+        state = null;
+    }
+
+    public AggregateState(Object obj) {
+        state = obj;
+    }
+
+    public void reset() {
+        state = null;
+    }
+
+    public void close() {
+        state = null;
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/DeserializedPreclusteredGroupOperator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/DeserializedPreclusteredGroupOperator.java
new file mode 100644
index 0000000..370f6d0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/DeserializedPreclusteredGroupOperator.java
@@ -0,0 +1,125 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.IComparator;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+
+public class DeserializedPreclusteredGroupOperator implements IOpenableDataWriterOperator {
+    private final int[] groupFields;
+
+    private final IComparator[] comparators;
+
+    private final IGroupAggregator aggregator;
+
+    private Object[] lastData;
+
+    private IOpenableDataWriter<Object[]> writer;
+
+    private List<Object[]> buffer;
+
+    private IOpenableDataReader<Object[]> reader;
+
+    public DeserializedPreclusteredGroupOperator(int[] groupFields, IComparator[] comparators,
+            IGroupAggregator aggregator) {
+        this.groupFields = groupFields;
+        this.comparators = comparators;
+        this.aggregator = aggregator;
+        buffer = new ArrayList<Object[]>();
+        reader = new IOpenableDataReader<Object[]>() {
+            private int idx;
+
+            @Override
+            public void open() {
+                idx = 0;
+            }
+
+            @Override
+            public void close() {
+            }
+
+            @Override
+            public Object[] readData() {
+                return idx >= buffer.size() ? null : buffer.get(idx++);
+            }
+        };
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (!buffer.isEmpty()) {
+            aggregate();
+        }
+        writer.close();
+        try {
+            aggregator.close();
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    private void aggregate() throws HyracksDataException {
+        reader.open();
+        aggregator.aggregate(reader, writer);
+        reader.close();
+        buffer.clear();
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        lastData = null;
+        writer.open();
+    }
+
+    @Override
+    public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+        if (index != 0) {
+            throw new IllegalArgumentException();
+        }
+        this.writer = writer;
+    }
+
+    @Override
+    public void writeData(Object[] data) throws HyracksDataException {
+        if (lastData != null && compare(data, lastData) != 0) {
+            aggregate();
+        }
+        lastData = data;
+        buffer.add(data);
+    }
+
+    private int compare(Object[] d1, Object[] d2) {
+        for (int i = 0; i < groupFields.length; ++i) {
+            int fIdx = groupFields[i];
+            int c = comparators[i].compare(d1[fIdx], d2[fIdx]);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        // TODO Auto-generated method stub
+
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/FrameToolsForGroupers.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/FrameToolsForGroupers.java
new file mode 100644
index 0000000..2346160
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/FrameToolsForGroupers.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+/**
+ *
+ */
+public class FrameToolsForGroupers {
+
+    public static void writeFields(byte[] buf, int offset, int length, ArrayTupleBuilder tupleBuilder)
+            throws HyracksDataException {
+        writeFields(buf, offset, length, tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+                tupleBuilder.getSize());
+    }
+
+    public static void writeFields(byte[] buf, int offset, int length, int[] fieldsOffset, byte[] data, int dataOffset,
+            int dataLength) throws HyracksDataException {
+        if (dataLength + 4 * fieldsOffset.length > length) {
+            throw new HyracksDataException("Out of buffer bound: try to write too much data (" + dataLength
+                    + ") to the given bound (" + length + ").");
+        }
+
+        ByteBuffer buffer = ByteBuffer.wrap(buf, offset, length);
+        for (int i = 0; i < fieldsOffset.length; i++) {
+            buffer.putInt(fieldsOffset[i]);
+        }
+        buffer.put(data, dataOffset, dataLength);
+    }
+
+    public static void updateFrameMetaForNewTuple(ByteBuffer buffer, int addedTupleLength) throws HyracksDataException {
+        int currentTupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()));
+        int currentTupleEndOffset = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()) - 4
+                * currentTupleCount);
+        int newTupleEndOffset = currentTupleEndOffset + addedTupleLength;
+
+        // update tuple end offset
+        buffer.putInt(FrameHelper.getTupleCountOffset(buffer.capacity()) - 4 * (currentTupleCount + 1),
+                newTupleEndOffset);
+        // Update the tuple count
+        buffer.putInt(FrameHelper.getTupleCountOffset(buffer.capacity()), currentTupleCount + 1);
+    }
+
+    public static void updateFrameMetaForNewTuple(ByteBuffer buffer, int addedTupleLength, boolean isReset)
+            throws HyracksDataException {
+        int currentTupleCount;
+        int currentTupleEndOffset;
+        if (isReset) {
+            currentTupleCount = 0;
+            currentTupleEndOffset = 0;
+        } else {
+            currentTupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()));
+            currentTupleEndOffset = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()) - 4
+                    * currentTupleCount);
+        }
+        int newTupleEndOffset = currentTupleEndOffset + addedTupleLength;
+
+        // update tuple end offset
+        buffer.putInt(FrameHelper.getTupleCountOffset(buffer.capacity()) - 4 * (currentTupleCount + 1),
+                newTupleEndOffset);
+        // Update the tuple count
+        buffer.putInt(FrameHelper.getTupleCountOffset(buffer.capacity()), currentTupleCount + 1);
+    }
+
+    public static boolean isFrameOverflowing(ByteBuffer buffer, int length, boolean isReset)
+            throws HyracksDataException {
+
+        int currentTupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()));
+        if (currentTupleCount == 0 || isReset) {
+            return length + 4 + 4 > buffer.capacity();
+        }
+        int currentTupleEndOffset = buffer.getInt(FrameHelper.getTupleCountOffset(buffer.capacity()) - 4
+                * currentTupleCount);
+        return currentTupleEndOffset + length + 4 + (currentTupleCount + 1) * 4 > buffer.capacity();
+    }
+}
diff --git a/hyracks/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
new file mode 100644
index 0000000..f2b56fa
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -0,0 +1,493 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.nio.ByteBuffer;
+import 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.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+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;
+
+/**
+ *
+ */
+public class HashSpillableTableFactory implements ISpillableTableFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final ITuplePartitionComputerFactory tpcf;
+    private final int tableSize;
+
+    public HashSpillableTableFactory(ITuplePartitionComputerFactory tpcf, int tableSize) {
+        this.tpcf = tpcf;
+        this.tableSize = tableSize;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.dataflow.std.aggregations.ISpillableTableFactory#
+     * buildSpillableTable(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * int[], edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory[],
+     * edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory,
+     * edu.
+     * uci.ics.hyracks.dataflow.std.aggregations.IFieldAggregateDescriptorFactory
+     * [], edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int)
+     */
+    @Override
+    public ISpillableTable buildSpillableTable(final IHyracksTaskContext ctx, final int[] keyFields,
+            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+            IAggregatorDescriptorFactory aggregateFactory, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor, final int framesLimit) throws HyracksDataException {
+        final int[] storedKeys = new int[keyFields.length];
+        @SuppressWarnings("rawtypes")
+        ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[keyFields.length];
+        for (int i = 0; i < keyFields.length; i++) {
+            storedKeys[i] = i;
+            storedKeySerDeser[i] = inRecordDescriptor.getFields()[keyFields[i]];
+        }
+
+        RecordDescriptor internalRecordDescriptor = outRecordDescriptor;
+        final FrameTupleAccessor storedKeysAccessor1 = new FrameTupleAccessor(ctx.getFrameSize(),
+                internalRecordDescriptor);
+        final FrameTupleAccessor storedKeysAccessor2 = new FrameTupleAccessor(ctx.getFrameSize(),
+                internalRecordDescriptor);
+
+        final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+
+        final FrameTuplePairComparator ftpcPartial = new FrameTuplePairComparator(keyFields, storedKeys, comparators);
+
+        final FrameTuplePairComparator ftpcTuple = new FrameTuplePairComparator(storedKeys, storedKeys, comparators);
+
+        final ITuplePartitionComputer tpc = tpcf.createPartitioner();
+
+        final INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory
+                .createNormalizedKeyComputer();
+
+        int[] keyFieldsInPartialResults = new int[keyFields.length];
+        for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
+            keyFieldsInPartialResults[i] = i;
+        }
+
+        final IAggregatorDescriptor aggregator = aggregateFactory.createAggregator(ctx, inRecordDescriptor,
+                outRecordDescriptor, keyFields, keyFieldsInPartialResults);
+
+        final AggregateState aggregateState = aggregator.createAggregateStates();
+
+        final ArrayTupleBuilder stateTupleBuilder;
+        if (keyFields.length < outRecordDescriptor.getFields().length) {
+            stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+        } else {
+            stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length + 1);
+        }
+
+        final ArrayTupleBuilder outputTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+
+        return new ISpillableTable() {
+
+            private int lastBufIndex;
+
+            private ByteBuffer outputFrame;
+            private FrameTupleAppender outputAppender;
+
+            private FrameTupleAppender stateAppender = new FrameTupleAppender(ctx.getFrameSize());
+
+            private final ISerializableTable table = new SerializableHashTable(tableSize, ctx);
+            private final TuplePointer storedTuplePointer = new TuplePointer();
+            private final List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
+
+            /**
+             * A tuple is "pointed" to by 3 entries in the tPointers array. [0]
+             * = Frame index in the "Frames" list, [1] = Tuple index in the
+             * frame, [2] = Poor man's normalized key for the tuple.
+             */
+            private int[] tPointers;
+
+            @Override
+            public void sortFrames() {
+                int sfIdx = storedKeys[0];
+                int totalTCount = table.getTupleCount();
+                tPointers = new int[totalTCount * 3];
+                int ptr = 0;
+
+                for (int i = 0; i < tableSize; i++) {
+                    int entry = i;
+                    int offset = 0;
+                    do {
+                        table.getTuplePointer(entry, offset, storedTuplePointer);
+                        if (storedTuplePointer.frameIndex < 0)
+                            break;
+                        tPointers[ptr * 3] = entry;
+                        tPointers[ptr * 3 + 1] = offset;
+                        table.getTuplePointer(entry, offset, storedTuplePointer);
+                        int fIndex = storedTuplePointer.frameIndex;
+                        int tIndex = storedTuplePointer.tupleIndex;
+                        storedKeysAccessor1.reset(frames.get(fIndex));
+                        int tStart = storedKeysAccessor1.getTupleStartOffset(tIndex);
+                        int f0StartRel = storedKeysAccessor1.getFieldStartOffset(tIndex, sfIdx);
+                        int f0EndRel = storedKeysAccessor1.getFieldEndOffset(tIndex, sfIdx);
+                        int f0Start = f0StartRel + tStart + storedKeysAccessor1.getFieldSlotsLength();
+                        tPointers[ptr * 3 + 2] = nkc == null ? 0 : nkc.normalize(storedKeysAccessor1.getBuffer()
+                                .array(), f0Start, f0EndRel - f0StartRel);
+                        ptr++;
+                        offset++;
+                    } while (true);
+                }
+                /**
+                 * Sort using quick sort
+                 */
+                if (tPointers.length > 0) {
+                    sort(tPointers, 0, totalTCount);
+                }
+            }
+
+            @Override
+            public void reset() {
+                lastBufIndex = -1;
+                tPointers = null;
+                table.reset();
+                aggregator.reset();
+            }
+
+            @Override
+            public boolean insert(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+                if (lastBufIndex < 0)
+                    nextAvailableFrame();
+                int entry = tpc.partition(accessor, tIndex, tableSize);
+                boolean foundGroup = false;
+                int offset = 0;
+                do {
+                    table.getTuplePointer(entry, offset++, storedTuplePointer);
+                    if (storedTuplePointer.frameIndex < 0)
+                        break;
+                    storedKeysAccessor1.reset(frames.get(storedTuplePointer.frameIndex));
+                    int c = ftpcPartial.compare(accessor, tIndex, storedKeysAccessor1, storedTuplePointer.tupleIndex);
+                    if (c == 0) {
+                        foundGroup = true;
+                        break;
+                    }
+                } while (true);
+
+                if (!foundGroup) {
+
+                    stateTupleBuilder.reset();
+
+                    for (int k = 0; k < keyFields.length; k++) {
+                        stateTupleBuilder.addField(accessor, tIndex, keyFields[k]);
+                    }
+
+                    aggregator.init(stateTupleBuilder, accessor, tIndex, aggregateState);
+                    if (!stateAppender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
+                            stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
+                        if (!nextAvailableFrame()) {
+                            return false;
+                        }
+                        if (!stateAppender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
+                                stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
+                            throw new HyracksDataException("Cannot init external aggregate state in a frame.");
+                        }
+                    }
+
+                    storedTuplePointer.frameIndex = lastBufIndex;
+                    storedTuplePointer.tupleIndex = stateAppender.getTupleCount() - 1;
+                    table.insert(entry, storedTuplePointer);
+                } else {
+
+                    aggregator.aggregate(accessor, tIndex, storedKeysAccessor1, storedTuplePointer.tupleIndex,
+                            aggregateState);
+
+                }
+                return true;
+            }
+
+            @Override
+            public List<ByteBuffer> getFrames() {
+                return frames;
+            }
+
+            @Override
+            public int getFrameCount() {
+                return lastBufIndex;
+            }
+
+            @Override
+            public void flushFrames(IFrameWriter writer, boolean isPartial) throws HyracksDataException {
+                if (outputFrame == null) {
+                    outputFrame = ctx.allocateFrame();
+                }
+
+                if (outputAppender == null) {
+                    outputAppender = new FrameTupleAppender(outputFrame.capacity());
+                }
+
+                outputAppender.reset(outputFrame, true);
+
+                writer.open();
+
+                if (tPointers == null) {
+                    // Not sorted
+                    for (int i = 0; i < tableSize; ++i) {
+                        int entry = i;
+                        int offset = 0;
+                        do {
+                            table.getTuplePointer(entry, offset++, storedTuplePointer);
+                            if (storedTuplePointer.frameIndex < 0)
+                                break;
+                            int bIndex = storedTuplePointer.frameIndex;
+                            int tIndex = storedTuplePointer.tupleIndex;
+
+                            storedKeysAccessor1.reset(frames.get(bIndex));
+
+                            outputTupleBuilder.reset();
+                            for (int k = 0; k < storedKeys.length; k++) {
+                                outputTupleBuilder.addField(storedKeysAccessor1, tIndex, storedKeys[k]);
+                            }
+
+                            if (isPartial) {
+
+                                aggregator.outputPartialResult(outputTupleBuilder, storedKeysAccessor1, tIndex,
+                                        aggregateState);
+
+                            } else {
+
+                                aggregator.outputFinalResult(outputTupleBuilder, storedKeysAccessor1, tIndex,
+                                        aggregateState);
+                            }
+
+                            if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
+                                    outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
+                                FrameUtils.flushFrame(outputFrame, writer);
+                                outputAppender.reset(outputFrame, true);
+                                if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
+                                        outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
+                                    throw new HyracksDataException(
+                                            "The output item is too large to be fit into a frame.");
+                                }
+                            }
+
+                        } while (true);
+                    }
+                    if (outputAppender.getTupleCount() > 0) {
+                        FrameUtils.flushFrame(outputFrame, writer);
+                        outputAppender.reset(outputFrame, true);
+                    }
+                    aggregator.close();
+                    return;
+                }
+                int n = tPointers.length / 3;
+                for (int ptr = 0; ptr < n; ptr++) {
+                    int tableIndex = tPointers[ptr * 3];
+                    int rowIndex = tPointers[ptr * 3 + 1];
+                    table.getTuplePointer(tableIndex, rowIndex, storedTuplePointer);
+                    int frameIndex = storedTuplePointer.frameIndex;
+                    int tupleIndex = storedTuplePointer.tupleIndex;
+                    // Get the frame containing the value
+                    ByteBuffer buffer = frames.get(frameIndex);
+                    storedKeysAccessor1.reset(buffer);
+
+                    outputTupleBuilder.reset();
+                    for (int k = 0; k < storedKeys.length; k++) {
+                        outputTupleBuilder.addField(storedKeysAccessor1, tupleIndex, storedKeys[k]);
+                    }
+
+                    if (isPartial) {
+
+                        aggregator.outputPartialResult(outputTupleBuilder, storedKeysAccessor1, tupleIndex,
+                                aggregateState);
+
+                    } else {
+
+                        aggregator.outputFinalResult(outputTupleBuilder, storedKeysAccessor1, tupleIndex,
+                                aggregateState);
+                    }
+
+                    if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
+                            outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
+                        FrameUtils.flushFrame(outputFrame, writer);
+                        outputAppender.reset(outputFrame, true);
+                        if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
+                                outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
+                            throw new HyracksDataException("The output item is too large to be fit into a frame.");
+                        }
+                    }
+                }
+                if (outputAppender.getTupleCount() > 0) {
+                    FrameUtils.flushFrame(outputFrame, writer);
+                    outputAppender.reset(outputFrame, true);
+                }
+                aggregator.close();
+            }
+
+            @Override
+            public void close() {
+                lastBufIndex = -1;
+                tPointers = null;
+                table.close();
+                frames.clear();
+                aggregateState.close();
+            }
+
+            /**
+             * Set the working frame to the next available frame in the frame
+             * list. There are two cases:<br>
+             * 1) If the next frame is not initialized, allocate a new frame. 2)
+             * When frames are already created, they are recycled.
+             * 
+             * @return Whether a new frame is added successfully.
+             */
+            private boolean nextAvailableFrame() {
+                // Return false if the number of frames is equal to the limit.
+                if (lastBufIndex + 1 >= framesLimit)
+                    return false;
+
+                if (frames.size() < framesLimit) {
+                    // Insert a new frame
+                    ByteBuffer frame = ctx.allocateFrame();
+                    frame.position(0);
+                    frame.limit(frame.capacity());
+                    frames.add(frame);
+                    stateAppender.reset(frame, true);
+                    lastBufIndex = frames.size() - 1;
+                } else {
+                    // Reuse an old frame
+                    lastBufIndex++;
+                    ByteBuffer frame = frames.get(lastBufIndex);
+                    frame.position(0);
+                    frame.limit(frame.capacity());
+                    stateAppender.reset(frame, true);
+                }
+                return true;
+            }
+
+            private void sort(int[] tPointers, int offset, int length) {
+                int m = offset + (length >> 1);
+                int mTable = tPointers[m * 3];
+                int mRow = tPointers[m * 3 + 1];
+                int mNormKey = tPointers[m * 3 + 2];
+
+                table.getTuplePointer(mTable, mRow, storedTuplePointer);
+                int mFrame = storedTuplePointer.frameIndex;
+                int mTuple = storedTuplePointer.tupleIndex;
+                storedKeysAccessor1.reset(frames.get(mFrame));
+
+                int a = offset;
+                int b = a;
+                int c = offset + length - 1;
+                int d = c;
+                while (true) {
+                    while (b <= c) {
+                        int bTable = tPointers[b * 3];
+                        int bRow = tPointers[b * 3 + 1];
+                        int bNormKey = tPointers[b * 3 + 2];
+                        int cmp = 0;
+                        if (bNormKey != mNormKey) {
+                            cmp = ((((long) bNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1 : 1;
+                        } else {
+                            table.getTuplePointer(bTable, bRow, storedTuplePointer);
+                            int bFrame = storedTuplePointer.frameIndex;
+                            int bTuple = storedTuplePointer.tupleIndex;
+                            storedKeysAccessor2.reset(frames.get(bFrame));
+                            cmp = ftpcTuple.compare(storedKeysAccessor2, bTuple, storedKeysAccessor1, mTuple);
+                        }
+                        if (cmp > 0) {
+                            break;
+                        }
+                        if (cmp == 0) {
+                            swap(tPointers, a++, b);
+                        }
+                        ++b;
+                    }
+                    while (c >= b) {
+                        int cTable = tPointers[c * 3];
+                        int cRow = tPointers[c * 3 + 1];
+                        int cNormKey = tPointers[c * 3 + 2];
+                        int cmp = 0;
+                        if (cNormKey != mNormKey) {
+                            cmp = ((((long) cNormKey) & 0xffffffffL) < (((long) mNormKey) & 0xffffffffL)) ? -1 : 1;
+                        } else {
+                            table.getTuplePointer(cTable, cRow, storedTuplePointer);
+                            int cFrame = storedTuplePointer.frameIndex;
+                            int cTuple = storedTuplePointer.tupleIndex;
+                            storedKeysAccessor2.reset(frames.get(cFrame));
+                            cmp = ftpcTuple.compare(storedKeysAccessor2, cTuple, storedKeysAccessor1, mTuple);
+                        }
+                        if (cmp < 0) {
+                            break;
+                        }
+                        if (cmp == 0) {
+                            swap(tPointers, c, d--);
+                        }
+                        --c;
+                    }
+                    if (b > c)
+                        break;
+                    swap(tPointers, b++, c--);
+                }
+
+                int s;
+                int n = offset + length;
+                s = Math.min(a - offset, b - a);
+                vecswap(tPointers, offset, b - s, s);
+                s = Math.min(d - c, n - d - 1);
+                vecswap(tPointers, b, n - s, s);
+
+                if ((s = b - a) > 1) {
+                    sort(tPointers, offset, s);
+                }
+                if ((s = d - c) > 1) {
+                    sort(tPointers, n - s, s);
+                }
+            }
+
+            private void swap(int x[], int a, int b) {
+                for (int i = 0; i < 3; ++i) {
+                    int t = x[a * 3 + i];
+                    x[a * 3 + i] = x[b * 3 + i];
+                    x[b * 3 + i] = t;
+                }
+            }
+
+            private void vecswap(int x[], int a, int b, int n) {
+                for (int i = 0; i < n; i++, a++, b++) {
+                    swap(x, a, b);
+                }
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptor.java
new file mode 100644
index 0000000..2cf978c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptor.java
@@ -0,0 +1,107 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+
+/**
+ *
+ */
+public interface IAggregatorDescriptor {
+
+    /**
+     * Create an aggregate state
+     * 
+     * @return
+     */
+    public AggregateState createAggregateStates();
+
+    /**
+     * Initialize the state based on the input tuple.
+     * 
+     * @param accessor
+     * @param tIndex
+     * @param fieldOutput
+     *            The data output for the frame containing the state. This may
+     *            be null, if the state is maintained as a java object
+     * @param state
+     *            The state to be initialized.
+     * @throws HyracksDataException
+     */
+    public void init(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex, AggregateState state)
+            throws HyracksDataException;
+
+    /**
+     * Reset the aggregator. The corresponding aggregate state should be reset
+     * too. Note that here the frame is not an input argument, since it can be
+     * reset outside of the aggregator (simply reset the starting index of the
+     * buffer).
+     * 
+     * @param state
+     */
+    public void reset();
+
+    /**
+     * Aggregate the value. Aggregate state should be updated correspondingly.
+     * 
+     * @param accessor
+     * @param tIndex
+     * @param data
+     *            The buffer containing the state, if frame-based-state is used.
+     *            This means that it can be null if java-object-based-state is
+     *            used.
+     * @param offset
+     * @param state
+     *            The aggregate state.
+     * @throws HyracksDataException
+     */
+    public void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor,
+            int stateTupleIndex, AggregateState state) throws HyracksDataException;
+
+    /**
+     * Output the partial aggregation result.
+     * 
+     * @param fieldOutput
+     *            The data output for the output frame
+     * @param data
+     *            The buffer containing the aggregation state
+     * @param offset
+     * @param state
+     *            The aggregation state.
+     * @throws HyracksDataException
+     */
+    public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            AggregateState state) throws HyracksDataException;
+
+    /**
+     * Output the final aggregation result.
+     * 
+     * @param fieldOutput
+     *            The data output for the output frame
+     * @param data
+     *            The buffer containing the aggregation state
+     * @param offset
+     * @param state
+     *            The aggregation state.
+     * @throws HyracksDataException
+     */
+    public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+            AggregateState state) throws HyracksDataException;
+
+    public void close();
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptorFactory.java
new file mode 100644
index 0000000..9ff915d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAggregatorDescriptorFactory.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ *
+ */
+public interface IAggregatorDescriptorFactory extends Serializable {
+
+    IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor, int[] keyFields, final int[] keyFieldsInPartialResults)
+            throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptor.java
new file mode 100644
index 0000000..a34fa73
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptor.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ *
+ */
+public interface IFieldAggregateDescriptor {
+
+    /**
+     * Initialize the state based on the input tuple.
+     * 
+     * @param accessor
+     * @param tIndex
+     * @param fieldOutput
+     *            The data output for the frame containing the state. This may
+     *            be null, if the state is maintained as a java object.
+     * 
+     *            Note that we have an assumption that the initialization of the
+     *            binary state (if any) inserts the state fields into the buffer
+     *            in a appending fashion. This means that an arbitrary initial
+     *            size of the state can be accquired.
+     * @param state
+     *            The state to be initialized.
+     * @throws HyracksDataException
+     */
+    public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+            throws HyracksDataException;
+
+    /**
+     * Reset the aggregator. The corresponding aggregate state should be reset
+     * too. Note that here the frame is not an input argument, since it can be
+     * reset outside of the aggregator (simply reset the starting index of the
+     * buffer).
+     * 
+     * @param state
+     */
+    public void reset();
+
+    /**
+     * Aggregate the value. Aggregate state should be updated correspondingly.
+     * 
+     * @param accessor
+     * @param tIndex
+     * @param data
+     *            The buffer containing the state, if frame-based-state is used.
+     *            This means that it can be null if java-object-based-state is
+     *            used.
+     * 
+     *            Here the length of binary state can be obtains from the state
+     *            parameter, and if the content to be filled into that is over-
+     *            flowing (larger than the reversed space), error should be
+     *            emit.
+     * @param offset
+     * @param state
+     *            The aggregate state.
+     * @throws HyracksDataException
+     */
+    public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset, AggregateState state)
+            throws HyracksDataException;
+
+    /**
+     * Output the partial aggregation result.
+     * 
+     * @param fieldOutput
+     *            The data output for the output frame
+     * @param data
+     *            The buffer containing the aggregation state
+     * @param offset
+     * @param state
+     *            The aggregation state.
+     * @throws HyracksDataException
+     */
+    public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+            throws HyracksDataException;
+
+    /**
+     * Output the final aggregation result.
+     * 
+     * @param fieldOutput
+     *            The data output for the output frame
+     * @param data
+     *            The buffer containing the aggregation state
+     * @param offset
+     * @param state
+     *            The aggregation state.
+     * @throws HyracksDataException
+     */
+    public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+            throws HyracksDataException;
+
+    public boolean needsBinaryState();
+
+    public boolean needsObjectState();
+
+    public AggregateState createState();
+
+    /**
+     * Close the field aggregator
+     */
+    public void close();
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptorFactory.java
new file mode 100644
index 0000000..6f50597
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IFieldAggregateDescriptorFactory.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ *
+ */
+public interface IFieldAggregateDescriptorFactory extends Serializable {
+
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IGroupAggregator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IGroupAggregator.java
new file mode 100644
index 0000000..7e2ba4b
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IGroupAggregator.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataReader;
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IGroupAggregator extends Serializable {
+    public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException;
+
+    public void close() throws Exception;
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..ff80a23
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.nio.ByteBuffer;
+import 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.comm.io.FrameTupleAccessor;
+
+public interface ISpillableTable {
+
+    public void close();
+
+    public void reset();
+
+    public int getFrameCount();
+
+    public List<ByteBuffer> getFrames();
+
+    public void sortFrames();
+
+    public boolean insert(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException;
+
+    public void flushFrames(IFrameWriter writer, boolean isPartial) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
new file mode 100644
index 0000000..de9fac5
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group;
+
+import java.io.Serializable;
+
+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;
+
+public interface ISpillableTableFactory extends Serializable {
+    ISpillableTable buildSpillableTable(IHyracksTaskContext ctx, int[] keyFields,
+            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory normalizedKeyComputerFactory,
+            IAggregatorDescriptorFactory aggregateFactory, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor, int framesLimit) throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
new file mode 100644
index 0000000..2e781b5
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class AvgFieldGroupAggregatorFactory implements IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int aggField;
+
+    private final boolean useObjectState;
+
+    public AvgFieldGroupAggregatorFactory(int aggField, boolean useObjectState) {
+        this.aggField = aggField;
+        this.useObjectState = useObjectState;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.dataflow.std.aggregations.
+     * IFieldAggregateDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+
+        return new IFieldAggregateDescriptor() {
+
+            @Override
+            public void reset() {
+            }
+
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum, count;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                    count = IntegerSerializerDeserializer.getInt(data, offset + 4);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum = fields[0];
+                    count = fields[1];
+                }
+                try {
+                    fieldOutput.writeInt(sum);
+                    fieldOutput.writeInt(count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum, count;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                    count = IntegerSerializerDeserializer.getInt(data, offset + 4);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum = fields[0];
+                    count = fields[1];
+                }
+                try {
+                    fieldOutput.writeFloat((float) sum / count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+                int sum = 0;
+                int count = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+                count += 1;
+                if (!useObjectState) {
+                    try {
+                        fieldOutput.writeInt(sum);
+                        fieldOutput.writeInt(count);
+                    } catch (IOException e) {
+                        throw new HyracksDataException("I/O exception when initializing the aggregator.");
+                    }
+                } else {
+                    state.state = new Integer[] { sum, count };
+                }
+            }
+
+            @Override
+            public void close() {
+                // TODO Auto-generated method stub
+
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+                    AggregateState state) throws HyracksDataException {
+                int sum = 0, count = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+                count += 1;
+                if (!useObjectState) {
+                    ByteBuffer buf = ByteBuffer.wrap(data);
+                    sum += buf.getInt(offset);
+                    count += buf.getInt(offset + 4);
+                    buf.putInt(offset, sum);
+                    buf.putInt(offset + 4, count);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum += fields[0];
+                    count += fields[1];
+                    state.state = new Integer[] { sum, count };
+                }
+            }
+
+            @Override
+            public boolean needsObjectState() {
+                return useObjectState;
+            }
+
+            @Override
+            public boolean needsBinaryState() {
+                return !useObjectState;
+            }
+
+            @Override
+            public AggregateState createState() {
+                return new AggregateState(new Integer[] { 0, 0 });
+            }
+
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
new file mode 100644
index 0000000..cc5c1e1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class AvgFieldMergeAggregatorFactory implements IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int aggField;
+
+    private final boolean useObjectState;
+
+    public AvgFieldMergeAggregatorFactory(int aggField, boolean useObjectState) {
+        this.aggField = aggField;
+        this.useObjectState = useObjectState;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.dataflow.std.aggregations.
+     * IFieldAggregateDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+
+        return new IFieldAggregateDescriptor() {
+
+            @Override
+            public void reset() {
+            }
+
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum, count;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                    count = IntegerSerializerDeserializer.getInt(data, offset + 4);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum = fields[0];
+                    count = fields[1];
+                }
+                try {
+                    fieldOutput.writeInt(sum);
+                    fieldOutput.writeInt(count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum, count;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                    count = IntegerSerializerDeserializer.getInt(data, offset + 4);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum = fields[0];
+                    count = fields[1];
+                }
+                try {
+                    fieldOutput.writeFloat((float) sum / count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void close() {
+                // TODO Auto-generated method stub
+
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+                    AggregateState state) throws HyracksDataException {
+                int sum = 0, count = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+                count += 1;
+                if (!useObjectState) {
+                    ByteBuffer buf = ByteBuffer.wrap(data);
+                    sum += buf.getInt(offset);
+                    count += buf.getInt(offset + 4);
+                    buf.putInt(offset, sum);
+                    buf.putInt(offset + 4, count);
+                } else {
+                    Integer[] fields = (Integer[]) state.state;
+                    sum += fields[0];
+                    count += fields[1];
+                    state.state = new Integer[] { sum, count };
+                }
+            }
+
+            @Override
+            public boolean needsObjectState() {
+                return useObjectState;
+            }
+
+            @Override
+            public boolean needsBinaryState() {
+                return !useObjectState;
+            }
+
+            @Override
+            public AggregateState createState() {
+                return new AggregateState(new Integer[] { 0, 0 });
+            }
+
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+                int sum = 0;
+                int count = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+                count += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart + 4);
+                if (!useObjectState) {
+                    try {
+                        fieldOutput.writeInt(sum);
+                        fieldOutput.writeInt(count);
+                    } catch (IOException e) {
+                        throw new HyracksDataException("I/O exception when initializing the aggregator.");
+                    }
+                } else {
+                    state.state = new Integer[] { sum, count };
+                }
+            }
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
new file mode 100644
index 0000000..9bfec8e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
@@ -0,0 +1,141 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class CountFieldAggregatorFactory implements IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final boolean useObjectState;
+
+    public CountFieldAggregatorFactory(boolean useObjectState) {
+        this.useObjectState = useObjectState;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.dataflow.std.aggregations.
+     * IFieldAggregateDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+        return new IFieldAggregateDescriptor() {
+
+            @Override
+            public void reset() {
+            }
+
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int count;
+                if (!useObjectState) {
+                    count = IntegerSerializerDeserializer.getInt(data, offset);
+                } else {
+                    count = (Integer) state.state;
+                }
+                try {
+                    fieldOutput.writeInt(count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int count;
+                if (!useObjectState) {
+                    count = IntegerSerializerDeserializer.getInt(data, offset);
+                } else {
+                    count = (Integer) state.state;
+                }
+                try {
+                    fieldOutput.writeInt(count);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+                int count = 1;
+                if (!useObjectState) {
+                    try {
+                        fieldOutput.writeInt(count);
+                    } catch (IOException e) {
+                        throw new HyracksDataException("I/O exception when initializing the aggregator.");
+                    }
+                } else {
+                    state.state = count;
+                }
+            }
+
+            public boolean needsObjectState() {
+                return useObjectState;
+            }
+
+            public boolean needsBinaryState() {
+                return !useObjectState;
+            }
+
+            public AggregateState createState() {
+                return new AggregateState(new Integer(0));
+            }
+
+            @Override
+            public void close() {
+
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+                    AggregateState state) throws HyracksDataException {
+                int count = 1;
+                if (!useObjectState) {
+                    ByteBuffer buf = ByteBuffer.wrap(data);
+                    count += buf.getInt(offset);
+                    buf.putInt(offset, count);
+                } else {
+                    count += (Integer) state.state;
+                    state.state = count;
+                }
+            }
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
new file mode 100644
index 0000000..f8b1d74
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
@@ -0,0 +1,158 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class FloatSumFieldAggregatorFactory implements
+        IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int aggField;
+
+    private final boolean useObjectState;
+    
+    public FloatSumFieldAggregatorFactory(int aggField, boolean useObjState){
+        this.aggField = aggField;
+        this.useObjectState = useObjState;
+    }
+    
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory#createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext, edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx,
+            RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+        return new IFieldAggregateDescriptor() {
+            
+            @Override
+            public void reset() {
+                
+            }
+            
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data,
+                    int offset, AggregateState state) throws HyracksDataException {
+                float sum;
+                if (!useObjectState) {
+                    sum = FloatSerializerDeserializer.getFloat(data, offset);
+                } else {
+                    sum = (Float) state.state;
+                }
+                try {
+                    fieldOutput.writeFloat(sum);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+            
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data,
+                    int offset, AggregateState state) throws HyracksDataException {
+                float sum;
+                if (!useObjectState) {
+                    sum = FloatSerializerDeserializer.getFloat(data, offset);
+                } else {
+                    sum = (Float) state.state;
+                }
+                try {
+                    fieldOutput.writeFloat(sum);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+            
+            @Override
+            public boolean needsObjectState() {
+                return useObjectState;
+            }
+            
+            @Override
+            public boolean needsBinaryState() {
+                return !useObjectState;
+            }
+            
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex,
+                    DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+                float sum = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+
+                sum += FloatSerializerDeserializer.getFloat(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+
+                if (!useObjectState) {
+                    try {
+                        fieldOutput.writeFloat(sum);
+                    } catch (IOException e) {
+                        throw new HyracksDataException("I/O exception when initializing the aggregator.");
+                    }
+                } else {
+                    state.state = sum;
+                }
+            }
+            
+            @Override
+            public AggregateState createState() {
+                return new AggregateState(new Float(0.0));
+            }
+            
+            @Override
+            public void close() {
+                // TODO Auto-generated method stub
+                
+            }
+            
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex,
+                    byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                float sum = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += FloatSerializerDeserializer.getFloat(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+
+                if (!useObjectState) {
+                    ByteBuffer buf = ByteBuffer.wrap(data);
+                    sum += buf.getFloat(offset);
+                    buf.putFloat(offset, sum);
+                } else {
+                    sum += (Float) state.state;
+                    state.state = sum;
+                }
+            }
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
new file mode 100644
index 0000000..7d85deb
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
@@ -0,0 +1,157 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class IntSumFieldAggregatorFactory implements IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int aggField;
+
+    private final boolean useObjectState;
+
+    public IntSumFieldAggregatorFactory(int aggField, boolean useObjState) {
+        this.aggField = aggField;
+        this.useObjectState = useObjState;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see edu.uci.ics.hyracks.dataflow.std.aggregations.
+     * IFieldAggregateDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+
+        return new IFieldAggregateDescriptor() {
+
+            @Override
+            public void reset() {
+            }
+
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                } else {
+                    sum = (Integer) state.state;
+                }
+                try {
+                    fieldOutput.writeInt(sum);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                int sum;
+                if (!useObjectState) {
+                    sum = IntegerSerializerDeserializer.getInt(data, offset);
+                } else {
+                    sum = (Integer) state.state;
+                }
+                try {
+                    fieldOutput.writeInt(sum);
+                } catch (IOException e) {
+                    throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
+                }
+            }
+
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+
+                int sum = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+
+                if (!useObjectState) {
+                    try {
+                        fieldOutput.writeInt(sum);
+                    } catch (IOException e) {
+                        throw new HyracksDataException("I/O exception when initializing the aggregator.");
+                    }
+                } else {
+                    state.state = sum;
+                }
+            }
+
+            public boolean needsObjectState() {
+                return useObjectState;
+            }
+
+            public boolean needsBinaryState() {
+                return !useObjectState;
+            }
+
+            public AggregateState createState() {
+                return new AggregateState(new Integer(0));
+            }
+
+            @Override
+            public void close() {
+
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+                    AggregateState state) throws HyracksDataException {
+                int sum = 0;
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                sum += IntegerSerializerDeserializer.getInt(accessor.getBuffer().array(),
+                        tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+
+                if (!useObjectState) {
+                    ByteBuffer buf = ByteBuffer.wrap(data);
+                    sum += buf.getInt(offset);
+                    buf.putInt(offset, sum);
+                } else {
+                    sum += (Integer) state.state;
+                    state.state = sum;
+                }
+            }
+        };
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MinMaxStringFieldAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MinMaxStringFieldAggregatorFactory.java
new file mode 100644
index 0000000..94ebcbd
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MinMaxStringFieldAggregatorFactory.java
@@ -0,0 +1,199 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class MinMaxStringFieldAggregatorFactory implements IFieldAggregateDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int aggField;
+
+    private final boolean isMax;
+
+    private final boolean hasBinaryState;
+
+    public MinMaxStringFieldAggregatorFactory(int aggField, boolean isMax, boolean hasBinaryState) {
+        this.aggField = aggField;
+        this.isMax = isMax;
+        this.hasBinaryState = hasBinaryState;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.dataflow.std.aggregators.IAggregatorDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor, int[])
+     */
+    @Override
+    public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+        return new IFieldAggregateDescriptor() {
+
+            @Override
+            public void reset() {
+            }
+
+            @Override
+            public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                try {
+                    if (hasBinaryState) {
+                        int stateIdx = IntegerSerializerDeserializer.getInt(data, offset);
+                        Object[] storedState = (Object[]) state.state;
+                        fieldOutput.writeUTF((String) storedState[stateIdx]);
+                    } else {
+                        fieldOutput.writeUTF((String) state.state);
+                    }
+                } catch (IOException e) {
+                    throw new HyracksDataException(
+                            "I/O exception when writing a string to the output writer in MinMaxStringAggregatorFactory.");
+                }
+            }
+
+            @Override
+            public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+                    throws HyracksDataException {
+                try {
+                    if (hasBinaryState) {
+                        int stateIdx = IntegerSerializerDeserializer.getInt(data, offset);
+                        Object[] storedState = (Object[]) state.state;
+                        fieldOutput.writeUTF((String) storedState[stateIdx]);
+                    } else {
+                        fieldOutput.writeUTF((String) state.state);
+                    }
+                } catch (IOException e) {
+                    throw new HyracksDataException(
+                            "I/O exception when writing a string to the output writer in MinMaxStringAggregatorFactory.");
+                }
+            }
+
+            @Override
+            public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
+                    throws HyracksDataException {
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                int fieldLength = accessor.getFieldLength(tIndex, aggField);
+                String strField = UTF8StringSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
+                        new ByteArrayInputStream(accessor.getBuffer().array(), tupleOffset
+                                + accessor.getFieldSlotsLength() + fieldStart, fieldLength)));
+                if (hasBinaryState) {
+                    // Object-binary-state
+                    Object[] storedState;
+                    if (state.state == null) {
+                        storedState = new Object[8];
+                        storedState[0] = new Integer(0);
+                        state.state = storedState;
+                    } else {
+                        storedState = (Object[]) state.state;
+                    }
+                    int stateCount = (Integer) (storedState[0]);
+                    if (stateCount + 1 >= storedState.length) {
+                        storedState = Arrays.copyOf(storedState, storedState.length * 2);
+                        state.state = storedState;
+                    }
+
+                    stateCount++;
+                    storedState[0] = stateCount;
+                    storedState[stateCount] = strField;
+                    try {
+                        fieldOutput.writeInt(stateCount);
+                    } catch (IOException e) {
+                        throw new HyracksDataException(e.fillInStackTrace());
+                    }
+                } else {
+                    // Only object-state
+                    state.state = strField;
+                }
+            }
+
+            @Override
+            public void close() {
+                // TODO Auto-generated method stub
+
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+                    AggregateState state) throws HyracksDataException {
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
+                int fieldLength = accessor.getFieldLength(tIndex, aggField);
+                String strField = UTF8StringSerializerDeserializer.INSTANCE.deserialize(new DataInputStream(
+                        new ByteArrayInputStream(accessor.getBuffer().array(), tupleOffset
+                                + accessor.getFieldSlotsLength() + fieldStart, fieldLength)));
+                if (hasBinaryState) {
+                    int stateIdx = IntegerSerializerDeserializer.getInt(data, offset);
+
+                    Object[] storedState = (Object[]) state.state;
+
+                    if (isMax) {
+                        if (strField.length() > ((String) (storedState[stateIdx])).length()) {
+                            storedState[stateIdx] = strField;
+                        }
+                    } else {
+                        if (strField.length() < ((String) (storedState[stateIdx])).length()) {
+                            storedState[stateIdx] = strField;
+                        }
+                    }
+                } else {
+                    if (isMax) {
+                        if (strField.length() > ((String) (state.state)).length()) {
+                            state.state = strField;
+                        }
+                    } else {
+                        if (strField.length() < ((String) (state.state)).length()) {
+                            state.state = strField;
+                        }
+                    }
+                }
+            }
+
+            public boolean needsObjectState() {
+                return true;
+            }
+
+            public boolean needsBinaryState() {
+                return hasBinaryState;
+            }
+
+            public AggregateState createState() {
+                return new AggregateState();
+            }
+
+        };
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java
new file mode 100644
index 0000000..6c436a9
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/aggregators/MultiFieldsAggregatorFactory.java
@@ -0,0 +1,170 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.aggregators;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
+
+/**
+ *
+ */
+public class MultiFieldsAggregatorFactory implements IAggregatorDescriptorFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final IFieldAggregateDescriptorFactory[] aggregatorFactories;
+    private int[] keys;
+
+    public MultiFieldsAggregatorFactory(int[] keys, IFieldAggregateDescriptorFactory[] aggregatorFactories) {
+        this.keys = keys;
+        this.aggregatorFactories = aggregatorFactories;
+    }
+
+    public MultiFieldsAggregatorFactory(IFieldAggregateDescriptorFactory[] aggregatorFactories) {
+        this.aggregatorFactories = aggregatorFactories;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.dataflow.std.aggregations.IAggregatorDescriptorFactory
+     * #createAggregator(edu.uci.ics.hyracks.api.context.IHyracksTaskContext,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor,
+     * edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor)
+     */
+    @Override
+    public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor, final int[] keyFields, final int[] keyFieldsInPartialResults)
+            throws HyracksDataException {
+
+        final IFieldAggregateDescriptor[] aggregators = new IFieldAggregateDescriptor[aggregatorFactories.length];
+        for (int i = 0; i < aggregators.length; i++) {
+            aggregators[i] = aggregatorFactories[i].createAggregator(ctx, inRecordDescriptor, outRecordDescriptor);
+        }
+
+        if (this.keys == null) {
+            this.keys = keyFields;
+        }
+
+        return new IAggregatorDescriptor() {
+
+            @Override
+            public void reset() {
+                for (int i = 0; i < aggregators.length; i++) {
+                    aggregators[i].reset();
+                }
+            }
+
+            @Override
+            public void outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                DataOutput dos = tupleBuilder.getDataOutput();
+
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                for (int i = 0; i < aggregators.length; i++) {
+                    int fieldOffset = accessor.getFieldStartOffset(tIndex, keys.length + i);
+                    aggregators[i].outputPartialResult(dos, accessor.getBuffer().array(),
+                            fieldOffset + accessor.getFieldSlotsLength() + tupleOffset,
+                            ((AggregateState[]) state.state)[i]);
+                    tupleBuilder.addFieldEndOffset();
+                }
+
+            }
+
+            @Override
+            public void outputFinalResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                DataOutput dos = tupleBuilder.getDataOutput();
+
+                int tupleOffset = accessor.getTupleStartOffset(tIndex);
+                for (int i = 0; i < aggregators.length; i++) {
+                    if (aggregators[i].needsBinaryState()) {
+                        int fieldOffset = accessor.getFieldStartOffset(tIndex, keys.length + i);
+                        aggregators[i].outputFinalResult(dos, accessor.getBuffer().array(),
+                                tupleOffset + accessor.getFieldSlotsLength() + fieldOffset,
+                                ((AggregateState[]) state.state)[i]);
+                    } else {
+                        aggregators[i].outputFinalResult(dos, null, 0, ((AggregateState[]) state.state)[i]);
+                    }
+                    tupleBuilder.addFieldEndOffset();
+                }
+            }
+
+            @Override
+            public void init(ArrayTupleBuilder tupleBuilder, IFrameTupleAccessor accessor, int tIndex,
+                    AggregateState state) throws HyracksDataException {
+                DataOutput dos = tupleBuilder.getDataOutput();
+
+                for (int i = 0; i < aggregators.length; i++) {
+                    aggregators[i].init(accessor, tIndex, dos, ((AggregateState[]) state.state)[i]);
+                    if (aggregators[i].needsBinaryState()) {
+                        tupleBuilder.addFieldEndOffset();
+                    }
+                }
+            }
+
+            @Override
+            public AggregateState createAggregateStates() {
+                AggregateState[] states = new AggregateState[aggregators.length];
+                for (int i = 0; i < states.length; i++) {
+                    states[i] = aggregators[i].createState();
+                }
+                return new AggregateState(states);
+            }
+
+            @Override
+            public void close() {
+                for (int i = 0; i < aggregators.length; i++) {
+                    aggregators[i].close();
+                }
+            }
+
+            @Override
+            public void aggregate(IFrameTupleAccessor accessor, int tIndex, IFrameTupleAccessor stateAccessor,
+                    int stateTupleIndex, AggregateState state) throws HyracksDataException {
+                if (stateAccessor != null) {
+                    int stateTupleOffset = stateAccessor.getTupleStartOffset(stateTupleIndex);
+                    int fieldIndex = 0;
+                    for (int i = 0; i < aggregators.length; i++) {
+                        if (aggregators[i].needsBinaryState()) {
+                            int stateFieldOffset = stateAccessor.getFieldStartOffset(stateTupleIndex, keys.length
+                                    + fieldIndex);
+                            aggregators[i].aggregate(accessor, tIndex, stateAccessor.getBuffer().array(),
+                                    stateTupleOffset + stateAccessor.getFieldSlotsLength() + stateFieldOffset,
+                                    ((AggregateState[]) state.state)[i]);
+                            fieldIndex++;
+                        } else {
+                            aggregators[i].aggregate(accessor, tIndex, null, 0, ((AggregateState[]) state.state)[i]);
+                        }
+                    }
+                } else {
+                    for (int i = 0; i < aggregators.length; i++) {
+                        aggregators[i].aggregate(accessor, tIndex, null, 0, ((AggregateState[]) state.state)[i]);
+                    }
+                }
+            }
+        };
+    }
+}
diff --git a/hyracks/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
new file mode 100644
index 0000000..ede6728
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.external;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+
+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.RunFileReader;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.ISpillableTable;
+import edu.uci.ics.hyracks.dataflow.std.group.ISpillableTableFactory;
+
+class ExternalGroupBuildOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
+    private final IHyracksTaskContext ctx;
+    private final Object stateId;
+    private final int[] keyFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final INormalizedKeyComputerFactory firstNormalizerFactory;
+    private final IAggregatorDescriptorFactory aggregatorFactory;
+    private final int framesLimit;
+    private final ISpillableTableFactory spillableTableFactory;
+    private final RecordDescriptor inRecordDescriptor;
+    private final RecordDescriptor outRecordDescriptor;
+    private final FrameTupleAccessor accessor;
+
+    private ExternalGroupState state;
+
+    ExternalGroupBuildOperatorNodePushable(IHyracksTaskContext ctx, Object stateId, int[] keyFields, int framesLimit,
+            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory firstNormalizerFactory,
+            IAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor, ISpillableTableFactory spillableTableFactory) {
+        this.ctx = ctx;
+        this.stateId = stateId;
+        this.framesLimit = framesLimit;
+        this.aggregatorFactory = aggregatorFactory;
+        this.keyFields = keyFields;
+        this.comparatorFactories = comparatorFactories;
+        this.firstNormalizerFactory = firstNormalizerFactory;
+        this.spillableTableFactory = spillableTableFactory;
+        this.inRecordDescriptor = inRecordDescriptor;
+        this.outRecordDescriptor = outRecordDescriptor;
+        this.accessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        state = new ExternalGroupState(ctx.getJobletContext().getJobId(), stateId);
+        state.setRuns(new LinkedList<RunFileReader>());
+        ISpillableTable table = spillableTableFactory.buildSpillableTable(ctx, keyFields, comparatorFactories,
+                firstNormalizerFactory, aggregatorFactory, inRecordDescriptor, outRecordDescriptor, framesLimit);
+        table.reset();
+        state.setSpillableTable(table);
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        accessor.reset(buffer);
+        int tupleCount = accessor.getTupleCount();
+        ISpillableTable gTable = state.getSpillableTable();
+        for (int i = 0; i < tupleCount; i++) {
+            /**
+             * If the group table is too large, flush the table into
+             * a run file.
+             */
+            if (!gTable.insert(accessor, i)) {
+                flushFramesToRun();
+                if (!gTable.insert(accessor, i))
+                    throw new HyracksDataException("Failed to insert a new buffer into the aggregate operator!");
+            }
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        //do nothing for failures
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        ISpillableTable gTable = state.getSpillableTable();
+        if (gTable.getFrameCount() >= 0) {
+            if (state.getRuns().size() > 0) {
+                /**
+                 * flush the memory into the run file.
+                 */
+                flushFramesToRun();
+                gTable.close();
+                gTable = null;
+            }
+        }
+        ctx.setStateObject(state);
+    }
+
+    private void flushFramesToRun() throws HyracksDataException {
+        FileReference runFile;
+        try {
+            runFile = ctx.getJobletContext().createManagedWorkspaceFile(
+                    ExternalGroupOperatorDescriptor.class.getSimpleName());
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+        RunFileWriter writer = new RunFileWriter(runFile, ctx.getIOManager());
+        writer.open();
+        ISpillableTable gTable = state.getSpillableTable();
+        try {
+            gTable.sortFrames();
+            gTable.flushFrames(writer, true);
+        } catch (Exception ex) {
+            throw new HyracksDataException(ex);
+        } finally {
+            writer.close();
+        }
+        gTable.reset();
+        state.getRuns().add(((RunFileWriter) writer).createReader());
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..297b75c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
@@ -0,0 +1,462 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.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.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.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.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.io.RunFileReader;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.ISpillableTable;
+import edu.uci.ics.hyracks.dataflow.std.util.ReferenceEntry;
+import edu.uci.ics.hyracks.dataflow.std.util.ReferencedPriorityQueue;
+
+class ExternalGroupMergeOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+    private final IHyracksTaskContext ctx;
+    private final Object stateId;
+    private final int[] keyFields;
+    private final IBinaryComparator[] comparators;
+    private final AggregateState aggregateState;
+    private final ArrayTupleBuilder tupleBuilder;
+    private final int[] storedKeys;
+    private final IAggregatorDescriptor aggregator;
+    private final boolean isOutputSorted;
+    private final int framesLimit;
+    private final RecordDescriptor outRecordDescriptor;
+    /**
+     * Input frames, one for each run file.
+     */
+    private List<ByteBuffer> inFrames;
+    /**
+     * Output frame.
+     */
+    private ByteBuffer outFrame, writerFrame;
+    private final FrameTupleAppender outAppender;
+    private FrameTupleAppender writerAppender;
+    private LinkedList<RunFileReader> runs;
+    private ExternalGroupState aggState;
+    private ArrayTupleBuilder finalTupleBuilder;
+    /**
+     * how many frames to be read ahead once
+     */
+    private int runFrameLimit = 1;
+    private int[] currentFrameIndexInRun;
+    private int[] currentRunFrames;
+    private final FrameTupleAccessor outFrameAccessor;
+
+    ExternalGroupMergeOperatorNodePushable(IHyracksTaskContext ctx, Object stateId,
+            IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
+            IAggregatorDescriptorFactory mergerFactory, boolean isOutputSorted, int framesLimit,
+            RecordDescriptor outRecordDescriptor) throws HyracksDataException {
+        this.stateId = stateId;
+        this.keyFields = keyFields;
+        comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        int[] keyFieldsInPartialResults = new int[keyFields.length];
+        for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
+            keyFieldsInPartialResults[i] = i;
+        }
+
+        aggregator = mergerFactory.createAggregator(ctx, outRecordDescriptor, outRecordDescriptor, keyFields,
+                keyFieldsInPartialResults);
+        aggregateState = aggregator.createAggregateStates();
+
+        storedKeys = new int[keyFields.length];
+        /**
+         * Get the list of the fields in the stored records.
+         */
+        for (int i = 0; i < keyFields.length; ++i) {
+            storedKeys[i] = i;
+        }
+
+        tupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+        this.ctx = ctx;
+        outAppender = new FrameTupleAppender(ctx.getFrameSize());
+        outFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDescriptor);
+        this.isOutputSorted = isOutputSorted;
+        this.framesLimit = framesLimit;
+        this.outRecordDescriptor = outRecordDescriptor;
+    }
+
+    public void initialize() throws HyracksDataException {
+        aggState = (ExternalGroupState) ctx.getStateObject(stateId);
+        runs = aggState.getRuns();
+        writer.open();
+        try {
+            if (runs.size() <= 0) {
+                ISpillableTable gTable = aggState.getSpillableTable();
+                if (gTable != null) {
+                    if (isOutputSorted)
+                        gTable.sortFrames();
+                    gTable.flushFrames(writer, false);
+                }
+                gTable = null;
+                aggState = null;
+            } else {
+                aggState = null;
+                runs = new LinkedList<RunFileReader>(runs);
+                inFrames = new ArrayList<ByteBuffer>();
+                outFrame = ctx.allocateFrame();
+                outAppender.reset(outFrame, true);
+                outFrameAccessor.reset(outFrame);
+                while (runs.size() > 0) {
+                    try {
+                        doPass(runs);
+                    } catch (Exception e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+                inFrames.clear();
+            }
+        } catch (Exception e) {
+            writer.fail();
+            throw new HyracksDataException(e);
+        } finally {
+            aggregateState.close();
+            writer.close();
+        }
+    }
+
+    private void doPass(LinkedList<RunFileReader> runs) throws HyracksDataException {
+        FileReference newRun = null;
+        IFrameWriter writer = this.writer;
+        boolean finalPass = false;
+
+        while (inFrames.size() + 2 < framesLimit) {
+            inFrames.add(ctx.allocateFrame());
+        }
+        int runNumber;
+        if (runs.size() + 2 <= framesLimit) {
+            finalPass = true;
+            runFrameLimit = (framesLimit - 2) / runs.size();
+            runNumber = runs.size();
+        } else {
+            runNumber = framesLimit - 2;
+            newRun = ctx.getJobletContext().createManagedWorkspaceFile(
+                    ExternalGroupOperatorDescriptor.class.getSimpleName());
+            writer = new RunFileWriter(newRun, ctx.getIOManager());
+            writer.open();
+        }
+        try {
+            currentFrameIndexInRun = new int[runNumber];
+            currentRunFrames = new int[runNumber];
+            /**
+             * Create file readers for each input run file, only for
+             * the ones fit into the inFrames
+             */
+            RunFileReader[] runFileReaders = new RunFileReader[runNumber];
+            FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
+            Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
+            ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), outRecordDescriptor,
+                    runNumber, comparator);
+            /**
+             * current tuple index in each run
+             */
+            int[] tupleIndices = new int[runNumber];
+
+            for (int i = 0; i < runNumber; i++) {
+                int runIndex = topTuples.peek().getRunid();
+                tupleIndices[runIndex] = 0;
+                // Load the run file
+                runFileReaders[runIndex] = runs.get(runIndex);
+                runFileReaders[runIndex].open();
+
+                currentRunFrames[runIndex] = 0;
+                currentFrameIndexInRun[runIndex] = runIndex * runFrameLimit;
+                for (int j = 0; j < runFrameLimit; j++) {
+                    int frameIndex = currentFrameIndexInRun[runIndex] + j;
+                    if (runFileReaders[runIndex].nextFrame(inFrames.get(frameIndex))) {
+                        tupleAccessors[frameIndex] = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDescriptor);
+                        tupleAccessors[frameIndex].reset(inFrames.get(frameIndex));
+                        currentRunFrames[runIndex]++;
+                        if (j == 0)
+                            setNextTopTuple(runIndex, tupleIndices, runFileReaders, tupleAccessors, topTuples);
+                    } else {
+                        break;
+                    }
+                }
+            }
+
+            /**
+             * Start merging
+             */
+            while (!topTuples.areRunsExhausted()) {
+                /**
+                 * Get the top record
+                 */
+                ReferenceEntry top = topTuples.peek();
+                int tupleIndex = top.getTupleIndex();
+                int runIndex = topTuples.peek().getRunid();
+                FrameTupleAccessor fta = top.getAccessor();
+
+                int currentTupleInOutFrame = outFrameAccessor.getTupleCount() - 1;
+                if (currentTupleInOutFrame < 0
+                        || compareFrameTuples(fta, tupleIndex, outFrameAccessor, currentTupleInOutFrame) != 0) {
+                    /**
+                     * Initialize the first output record Reset the
+                     * tuple builder
+                     */
+
+                    tupleBuilder.reset();
+
+                    for (int k = 0; k < storedKeys.length; k++) {
+                        tupleBuilder.addField(fta, tupleIndex, storedKeys[k]);
+                    }
+
+                    aggregator.init(tupleBuilder, fta, tupleIndex, aggregateState);
+
+                    if (!outAppender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(),
+                            tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
+                        flushOutFrame(writer, finalPass);
+                        if (!outAppender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(),
+                                tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
+                            throw new HyracksDataException(
+                                    "The partial result is too large to be initialized in a frame.");
+                        }
+                    }
+
+                } else {
+                    /**
+                     * if new tuple is in the same group of the
+                     * current aggregator do merge and output to the
+                     * outFrame
+                     */
+
+                    aggregator.aggregate(fta, tupleIndex, outFrameAccessor, currentTupleInOutFrame, aggregateState);
+
+                }
+                tupleIndices[runIndex]++;
+                setNextTopTuple(runIndex, tupleIndices, runFileReaders, tupleAccessors, topTuples);
+            }
+
+            if (outAppender.getTupleCount() > 0) {
+                flushOutFrame(writer, finalPass);
+                outAppender.reset(outFrame, true);
+            }
+
+            aggregator.close();
+
+            runs.subList(0, runNumber).clear();
+            /**
+             * insert the new run file into the beginning of the run
+             * file list
+             */
+            if (!finalPass) {
+                runs.add(0, ((RunFileWriter) writer).createReader());
+            }
+        } finally {
+            if (!finalPass) {
+                writer.close();
+            }
+        }
+    }
+
+    private void flushOutFrame(IFrameWriter writer, boolean isFinal) throws HyracksDataException {
+
+        if (finalTupleBuilder == null) {
+            finalTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+        }
+
+        if (writerFrame == null) {
+            writerFrame = ctx.allocateFrame();
+        }
+
+        if (writerAppender == null) {
+            writerAppender = new FrameTupleAppender(ctx.getFrameSize());
+            writerAppender.reset(writerFrame, true);
+        }
+
+        outFrameAccessor.reset(outFrame);
+
+        for (int i = 0; i < outFrameAccessor.getTupleCount(); i++) {
+
+            finalTupleBuilder.reset();
+
+            for (int k = 0; k < storedKeys.length; k++) {
+                finalTupleBuilder.addField(outFrameAccessor, i, storedKeys[k]);
+            }
+
+            if (isFinal) {
+
+                aggregator.outputFinalResult(finalTupleBuilder, outFrameAccessor, i, aggregateState);
+
+            } else {
+
+                aggregator.outputPartialResult(finalTupleBuilder, outFrameAccessor, i, aggregateState);
+            }
+
+            if (!writerAppender.appendSkipEmptyField(finalTupleBuilder.getFieldEndOffsets(),
+                    finalTupleBuilder.getByteArray(), 0, finalTupleBuilder.getSize())) {
+                FrameUtils.flushFrame(writerFrame, writer);
+                writerAppender.reset(writerFrame, 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);
+        }
+
+        outAppender.reset(outFrame, true);
+    }
+
+    private void setNextTopTuple(int runIndex, int[] tupleIndices, RunFileReader[] runCursors,
+            FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws HyracksDataException {
+        int runStart = runIndex * runFrameLimit;
+        boolean existNext = false;
+        if (tupleAccessors[currentFrameIndexInRun[runIndex]] == null || runCursors[runIndex] == null) {
+            /**
+             * run already closed
+             */
+            existNext = false;
+        } else if (currentFrameIndexInRun[runIndex] - runStart < currentRunFrames[runIndex] - 1) {
+            /**
+             * not the last frame for this run
+             */
+            existNext = true;
+            if (tupleIndices[runIndex] >= tupleAccessors[currentFrameIndexInRun[runIndex]].getTupleCount()) {
+                tupleIndices[runIndex] = 0;
+                currentFrameIndexInRun[runIndex]++;
+            }
+        } else if (tupleIndices[runIndex] < tupleAccessors[currentFrameIndexInRun[runIndex]].getTupleCount()) {
+            /**
+             * the last frame has expired
+             */
+            existNext = true;
+        } else {
+            /**
+             * If all tuples in the targeting frame have been
+             * checked.
+             */
+            tupleIndices[runIndex] = 0;
+            currentFrameIndexInRun[runIndex] = runStart;
+            /**
+             * read in batch
+             */
+            currentRunFrames[runIndex] = 0;
+            for (int j = 0; j < runFrameLimit; j++) {
+                int frameIndex = currentFrameIndexInRun[runIndex] + j;
+                if (runCursors[runIndex].nextFrame(inFrames.get(frameIndex))) {
+                    tupleAccessors[frameIndex].reset(inFrames.get(frameIndex));
+                    existNext = true;
+                    currentRunFrames[runIndex]++;
+                } else {
+                    break;
+                }
+            }
+        }
+
+        if (existNext) {
+            topTuples.popAndReplace(tupleAccessors[currentFrameIndexInRun[runIndex]], tupleIndices[runIndex]);
+        } else {
+            topTuples.pop();
+            closeRun(runIndex, runCursors, tupleAccessors);
+        }
+    }
+
+    /**
+     * Close the run file, and also the corresponding readers and
+     * input frame.
+     * 
+     * @param index
+     * @param runCursors
+     * @param tupleAccessor
+     * @throws HyracksDataException
+     */
+    private void closeRun(int index, RunFileReader[] runCursors, IFrameTupleAccessor[] tupleAccessor)
+            throws HyracksDataException {
+        if (runCursors[index] != null) {
+            runCursors[index].close();
+            runCursors[index] = null;
+            int frameOffset = index * runFrameLimit;
+            for (int j = 0; j < runFrameLimit; j++) {
+                tupleAccessor[frameOffset + j] = null;
+            }
+        }
+    }
+
+    private int compareFrameTuples(IFrameTupleAccessor fta1, int j1, IFrameTupleAccessor fta2, int j2) {
+        byte[] b1 = fta1.getBuffer().array();
+        byte[] b2 = fta2.getBuffer().array();
+        for (int f = 0; f < keyFields.length; ++f) {
+            int fIdx = f;
+            int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength() + fta1.getFieldStartOffset(j1, fIdx);
+            int l1 = fta1.getFieldLength(j1, fIdx);
+            int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength() + fta2.getFieldStartOffset(j2, fIdx);
+            int l2_start = fta2.getFieldStartOffset(j2, fIdx);
+            int l2_end = fta2.getFieldEndOffset(j2, fIdx);
+            int l2 = l2_end - l2_start;
+            int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+
+    private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
+        return new Comparator<ReferenceEntry>() {
+
+            @Override
+            public int compare(ReferenceEntry o1, ReferenceEntry o2) {
+                FrameTupleAccessor fta1 = (FrameTupleAccessor) o1.getAccessor();
+                FrameTupleAccessor fta2 = (FrameTupleAccessor) o2.getAccessor();
+                int j1 = o1.getTupleIndex();
+                int j2 = o2.getTupleIndex();
+                byte[] b1 = fta1.getBuffer().array();
+                byte[] b2 = fta2.getBuffer().array();
+                for (int f = 0; f < keyFields.length; ++f) {
+                    int fIdx = f;
+                    int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
+                            + fta1.getFieldStartOffset(j1, fIdx);
+                    int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
+                    int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
+                            + fta2.getFieldStartOffset(j2, fIdx);
+                    int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
+                    int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                    if (c != 0) {
+                        return c;
+                    }
+                }
+                return 0;
+            }
+
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
new file mode 100644
index 0000000..9fd45e9
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
@@ -0,0 +1,139 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.external;
+
+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.IBinaryComparatorFactory;
+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.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.group.ISpillableTableFactory;
+
+/**
+ *
+ */
+public class ExternalGroupOperatorDescriptor extends AbstractOperatorDescriptor {
+
+    private static final int AGGREGATE_ACTIVITY_ID = 0;
+
+    private static final int MERGE_ACTIVITY_ID = 1;
+
+    private static final long serialVersionUID = 1L;
+    private final int[] keyFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final INormalizedKeyComputerFactory firstNormalizerFactory;
+
+    private final IAggregatorDescriptorFactory aggregatorFactory;
+    private final IAggregatorDescriptorFactory mergerFactory;
+
+    private final int framesLimit;
+    private final ISpillableTableFactory spillableTableFactory;
+    private final boolean isOutputSorted;
+
+    public ExternalGroupOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keyFields, int framesLimit,
+            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory firstNormalizerFactory,
+            IAggregatorDescriptorFactory aggregatorFactory, IAggregatorDescriptorFactory mergerFactory,
+            RecordDescriptor recordDescriptor, ISpillableTableFactory spillableTableFactory, boolean isOutputSorted) {
+        super(spec, 1, 1);
+        this.framesLimit = framesLimit;
+        if (framesLimit <= 1) {
+            /**
+             * Minimum of 2 frames: 1 for input records, and 1 for output
+             * aggregation results.
+             */
+            throw new IllegalStateException("frame limit should at least be 2, but it is " + framesLimit + "!");
+        }
+        this.aggregatorFactory = aggregatorFactory;
+        this.mergerFactory = mergerFactory;
+        this.keyFields = keyFields;
+        this.comparatorFactories = comparatorFactories;
+        this.firstNormalizerFactory = firstNormalizerFactory;
+        this.spillableTableFactory = spillableTableFactory;
+        this.isOutputSorted = isOutputSorted;
+
+        /**
+         * Set the record descriptor. Note that since this operator is a unary
+         * operator, only the first record descriptor is used here.
+         */
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor#contributeActivities
+     * (edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder)
+     */
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        AggregateActivity aggregateAct = new AggregateActivity(new ActivityId(getOperatorId(), AGGREGATE_ACTIVITY_ID));
+        MergeActivity mergeAct = new MergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
+
+        builder.addActivity(this, aggregateAct);
+        builder.addSourceEdge(0, aggregateAct, 0);
+
+        builder.addActivity(this, mergeAct);
+        builder.addTargetEdge(0, mergeAct, 0);
+
+        builder.addBlockingEdge(aggregateAct, mergeAct);
+    }
+
+    private class AggregateActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public AggregateActivity(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+                throws HyracksDataException {
+            return new ExternalGroupBuildOperatorNodePushable(ctx, new TaskId(getActivityId(), partition), keyFields,
+                    framesLimit, comparatorFactories, firstNormalizerFactory, aggregatorFactory,
+                    recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), recordDescriptors[0],
+                    spillableTableFactory);
+        }
+    }
+
+    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)
+                throws HyracksDataException {
+            return new ExternalGroupMergeOperatorNodePushable(ctx, new TaskId(new ActivityId(getOperatorId(),
+                    AGGREGATE_ACTIVITY_ID), partition), comparatorFactories, keyFields, mergerFactory, isOutputSorted,
+                    framesLimit, recordDescriptors[0]);
+        }
+
+    }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupState.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupState.java
new file mode 100644
index 0000000..a3cf0e3
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupState.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.external;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.LinkedList;
+
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
+import edu.uci.ics.hyracks.dataflow.std.group.ISpillableTable;
+
+public class ExternalGroupState extends AbstractStateObject {
+    private LinkedList<RunFileReader> runs;
+
+    private ISpillableTable gTable;
+
+    public ExternalGroupState() {
+    }
+
+    ExternalGroupState(JobId jobId, Object id) {
+        super(jobId, id);
+    }
+
+    public LinkedList<RunFileReader> getRuns() {
+        return runs;
+    }
+
+    public void setRuns(LinkedList<RunFileReader> runs) {
+        this.runs = runs;
+    }
+
+    public ISpillableTable getSpillableTable() {
+        return gTable;
+    }
+
+    public void setSpillableTable(ISpillableTable gTable) {
+        this.gTable = gTable;
+    }
+
+    @Override
+    public void toBytes(DataOutput out) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void fromBytes(DataInput in) throws IOException {
+        throw new UnsupportedOperationException();
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..801e7c7
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
@@ -0,0 +1,254 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.hash;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+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.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+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;
+
+class GroupingHashTable {
+    /**
+     * The pointers in the link store 3 int values for each entry in the
+     * hashtable: (bufferIdx, tIndex, accumulatorIdx).
+     * 
+     * @author vinayakb
+     */
+    private static class Link {
+        private static final int INIT_POINTERS_SIZE = 9;
+
+        int[] pointers;
+        int size;
+
+        Link() {
+            pointers = new int[INIT_POINTERS_SIZE];
+            size = 0;
+        }
+
+        void add(int bufferIdx, int tIndex, int accumulatorIdx) {
+            while (size + 3 > pointers.length) {
+                pointers = Arrays.copyOf(pointers, pointers.length * 2);
+            }
+            pointers[size++] = bufferIdx;
+            pointers[size++] = tIndex;
+            pointers[size++] = accumulatorIdx;
+        }
+    }
+
+    private static final int INIT_AGG_STATE_SIZE = 8;
+    private final IHyracksTaskContext ctx;
+
+    private final List<ByteBuffer> buffers;
+    private final Link[] table;
+    /**
+     * Aggregate states: a list of states for all groups maintained in the main
+     * memory.
+     */
+    private AggregateState[] aggregateStates;
+    private int accumulatorSize;
+
+    private int lastBIndex;
+    private final int[] storedKeys;
+    private final int[] keys;
+    private final IBinaryComparator[] comparators;
+    private final FrameTuplePairComparator ftpc;
+    private final ITuplePartitionComputer tpc;
+    private final IAggregatorDescriptor aggregator;
+
+    private final FrameTupleAppender appender;
+
+    private final FrameTupleAccessor storedKeysAccessor;
+
+    private final ArrayTupleBuilder stateTupleBuilder, outputTupleBuilder;
+
+    GroupingHashTable(IHyracksTaskContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
+            ITuplePartitionComputerFactory tpcf, IAggregatorDescriptorFactory aggregatorFactory,
+            RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int tableSize)
+            throws HyracksDataException {
+        this.ctx = ctx;
+
+        buffers = new ArrayList<ByteBuffer>();
+        table = new Link[tableSize];
+
+        keys = fields;
+        storedKeys = new int[fields.length];
+        @SuppressWarnings("rawtypes")
+        ISerializerDeserializer[] storedKeySerDeser = new ISerializerDeserializer[fields.length];
+        for (int i = 0; i < fields.length; ++i) {
+            storedKeys[i] = i;
+            storedKeySerDeser[i] = inRecordDescriptor.getFields()[fields[i]];
+        }
+
+        comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        ftpc = new FrameTuplePairComparator(fields, storedKeys, comparators);
+        tpc = tpcf.createPartitioner();
+
+        int[] keyFieldsInPartialResults = new int[fields.length];
+        for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
+            keyFieldsInPartialResults[i] = i;
+        }
+
+        this.aggregator = aggregatorFactory.createAggregator(ctx, inRecordDescriptor, outRecordDescriptor, fields,
+                keyFieldsInPartialResults);
+
+        this.aggregateStates = new AggregateState[INIT_AGG_STATE_SIZE];
+        accumulatorSize = 0;
+
+        RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(storedKeySerDeser);
+        storedKeysAccessor = new FrameTupleAccessor(ctx.getFrameSize(), storedKeysRecordDescriptor);
+        lastBIndex = -1;
+
+        appender = new FrameTupleAppender(ctx.getFrameSize());
+
+        addNewBuffer();
+
+        if (fields.length < outRecordDescriptor.getFields().length) {
+            stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+        } else {
+            stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length + 1);
+        }
+        outputTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+    }
+
+    private void addNewBuffer() {
+        ByteBuffer buffer = ctx.allocateFrame();
+        buffer.position(0);
+        buffer.limit(buffer.capacity());
+        buffers.add(buffer);
+        appender.reset(buffer, true);
+        ++lastBIndex;
+    }
+
+    void insert(FrameTupleAccessor accessor, int tIndex) throws Exception {
+        int entry = tpc.partition(accessor, tIndex, table.length);
+        Link link = table[entry];
+        if (link == null) {
+            link = table[entry] = new Link();
+        }
+        int saIndex = -1;
+        for (int i = 0; i < link.size; i += 3) {
+            int sbIndex = link.pointers[i];
+            int stIndex = link.pointers[i + 1];
+            storedKeysAccessor.reset(buffers.get(sbIndex));
+            int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
+            if (c == 0) {
+                saIndex = link.pointers[i + 2];
+                break;
+            }
+        }
+        if (saIndex < 0) {
+            // Did not find the key. Insert a new entry.
+            saIndex = accumulatorSize++;
+            // Add keys
+
+            // Add aggregation fields
+            AggregateState newState = aggregator.createAggregateStates();
+
+            stateTupleBuilder.reset();
+            for (int k = 0; k < keys.length; k++) {
+                stateTupleBuilder.addField(accessor, tIndex, keys[k]);
+            }
+
+            aggregator.init(stateTupleBuilder, accessor, tIndex, newState);
+
+            if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
+                    stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
+                addNewBuffer();
+                if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
+                        stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
+                    throw new HyracksDataException("Cannot init the aggregate state in a single frame.");
+                }
+            }
+
+            if (accumulatorSize >= aggregateStates.length) {
+                aggregateStates = Arrays.copyOf(aggregateStates, aggregateStates.length * 2);
+            }
+
+            aggregateStates[saIndex] = newState;
+
+            link.add(lastBIndex, appender.getTupleCount() - 1, saIndex);
+
+        } else {
+            aggregator.aggregate(accessor, tIndex, null, 0, aggregateStates[saIndex]);
+        }
+    }
+
+    void write(IFrameWriter writer) throws HyracksDataException {
+        ByteBuffer buffer = ctx.allocateFrame();
+        appender.reset(buffer, true);
+
+        for (int i = 0; i < table.length; ++i) {
+            Link link = table[i];
+            if (link != null) {
+                for (int j = 0; j < link.size; j += 3) {
+                    int bIndex = link.pointers[j];
+                    int tIndex = link.pointers[j + 1];
+                    int aIndex = link.pointers[j + 2];
+                    ByteBuffer keyBuffer = buffers.get(bIndex);
+                    storedKeysAccessor.reset(keyBuffer);
+
+                    // copy keys
+                    outputTupleBuilder.reset();
+                    for (int k = 0; k < storedKeys.length; k++) {
+                        outputTupleBuilder.addField(storedKeysAccessor, tIndex, storedKeys[k]);
+                    }
+
+                    aggregator.outputFinalResult(outputTupleBuilder, storedKeysAccessor, tIndex,
+                            aggregateStates[aIndex]);
+
+                    if (!appender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
+                            outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
+                        writer.nextFrame(buffer);
+                        appender.reset(buffer, true);
+                        if (!appender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
+                                outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
+                            throw new HyracksDataException("Cannot write the aggregation output into a frame.");
+                        }
+                    }
+
+                }
+            }
+        }
+        if (appender.getTupleCount() != 0) {
+            writer.nextFrame(buffer);
+        }
+    }
+
+    void close() throws HyracksDataException {
+        for (AggregateState aState : aggregateStates) {
+            aState.close();
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..0b73766
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.hash;
+
+import java.nio.ByteBuffer;
+
+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.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+
+class HashGroupBuildOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
+    private final IHyracksTaskContext ctx;
+    private final FrameTupleAccessor accessor;
+    private final Object stateId;
+    private final int[] keys;
+    private final ITuplePartitionComputerFactory tpcf;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final IAggregatorDescriptorFactory aggregatorFactory;
+    private final int tableSize;
+    private final RecordDescriptor inRecordDescriptor;
+    private final RecordDescriptor outRecordDescriptor;
+
+    private HashGroupState state;
+
+    HashGroupBuildOperatorNodePushable(IHyracksTaskContext ctx, Object stateId, int[] keys,
+            ITuplePartitionComputerFactory tpcf, IBinaryComparatorFactory[] comparatorFactories,
+            IAggregatorDescriptorFactory aggregatorFactory, int tableSize, RecordDescriptor inRecordDescriptor,
+            RecordDescriptor outRecordDescriptor) {
+        this.ctx = ctx;
+        this.accessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
+        this.stateId = stateId;
+        this.keys = keys;
+        this.tpcf = tpcf;
+        this.comparatorFactories = comparatorFactories;
+        this.aggregatorFactory = aggregatorFactory;
+        this.tableSize = tableSize;
+        this.inRecordDescriptor = inRecordDescriptor;
+        this.outRecordDescriptor = outRecordDescriptor;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        state = new HashGroupState(ctx.getJobletContext().getJobId(), stateId);
+        state.setHashTable(new GroupingHashTable(ctx, keys, comparatorFactories, tpcf, aggregatorFactory,
+                inRecordDescriptor, outRecordDescriptor, tableSize));
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        GroupingHashTable table = state.getHashTable();
+        accessor.reset(buffer);
+        int tupleCount = accessor.getTupleCount();
+        for (int i = 0; i < tupleCount; ++i) {
+            try {
+                table.insert(accessor, i);
+            } catch (Exception e) {
+                System.out.println(e.toString());
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        ctx.setStateObject(state);
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        throw new HyracksDataException("HashGroupOperator is failed.");
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupOperatorDescriptor.java
new file mode 100644
index 0000000..034e13f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupOperatorDescriptor.java
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.hash;
+
+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.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+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.group.IAggregatorDescriptorFactory;
+
+/**
+ *
+ */
+public class HashGroupOperatorDescriptor extends AbstractOperatorDescriptor {
+
+    private static final int HASH_BUILD_ACTIVITY_ID = 0;
+
+    private static final int OUTPUT_ACTIVITY_ID = 1;
+
+    private static final long serialVersionUID = 1L;
+
+    private final int[] keys;
+    private final ITuplePartitionComputerFactory tpcf;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+
+    private final IAggregatorDescriptorFactory aggregatorFactory;
+
+    private final int tableSize;
+
+    public HashGroupOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys,
+            ITuplePartitionComputerFactory tpcf, IBinaryComparatorFactory[] comparatorFactories,
+            IAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor outRecordDescriptor, int tableSize) {
+        super(spec, 1, 1);
+        this.keys = keys;
+        this.tpcf = tpcf;
+        this.comparatorFactories = comparatorFactories;
+        this.aggregatorFactory = aggregatorFactory;
+        recordDescriptors[0] = outRecordDescriptor;
+        this.tableSize = tableSize;
+    }
+
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor#contributeActivities
+     * (edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder)
+     */
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        HashBuildActivity ha = new HashBuildActivity(new ActivityId(odId, HASH_BUILD_ACTIVITY_ID));
+        builder.addActivity(this, ha);
+
+        OutputActivity oa = new OutputActivity(new ActivityId(odId, OUTPUT_ACTIVITY_ID));
+        builder.addActivity(this, oa);
+
+        builder.addSourceEdge(0, ha, 0);
+        builder.addTargetEdge(0, oa, 0);
+        builder.addBlockingEdge(ha, oa);
+    }
+
+    private class HashBuildActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public HashBuildActivity(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            return new HashGroupBuildOperatorNodePushable(ctx, new TaskId(getActivityId(), partition), keys, tpcf,
+                    comparatorFactories, aggregatorFactory, tableSize, recordDescProvider.getInputRecordDescriptor(
+                            getActivityId(), 0), recordDescriptors[0]);
+        }
+    }
+
+    private class OutputActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public OutputActivity(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            return new HashGroupOutputOperatorNodePushable(ctx, new TaskId(new ActivityId(getOperatorId(),
+                    HASH_BUILD_ACTIVITY_ID), partition));
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupOutputOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupOutputOperatorNodePushable.java
new file mode 100644
index 0000000..9e3ada0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupOutputOperatorNodePushable.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.hash;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+
+class HashGroupOutputOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+    private final IHyracksTaskContext ctx;
+    private final Object stateId;
+
+    HashGroupOutputOperatorNodePushable(IHyracksTaskContext ctx, Object stateId) {
+        this.ctx = ctx;
+        this.stateId = stateId;
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+        HashGroupState buildState = (HashGroupState) ctx.getStateObject(stateId);
+        GroupingHashTable table = buildState.getHashTable();
+        writer.open();
+        try {
+            table.write(writer);
+        } catch (Exception e) {
+            writer.fail();
+            throw new HyracksDataException(e);
+        } finally {
+            writer.close();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupState.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupState.java
new file mode 100644
index 0000000..adaaaa6
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupState.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.hash;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
+
+public class HashGroupState extends AbstractStateObject {
+    private GroupingHashTable table;
+
+    public HashGroupState() {
+    }
+
+    HashGroupState(JobId jobId, Object id) {
+        super(jobId, id);
+    }
+
+    public GroupingHashTable getHashTable() {
+        return table;
+    }
+
+    public void setHashTable(GroupingHashTable table) {
+        this.table = table;
+    }
+
+    @Override
+    public void toBytes(DataOutput out) throws IOException {
+
+    }
+
+    @Override
+    public void fromBytes(DataInput in) throws IOException {
+
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorDescriptor.java
new file mode 100644
index 0000000..d37b818
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorDescriptor.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.preclustered;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+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.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+
+public class PreclusteredGroupOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private final int[] groupFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final IAggregatorDescriptorFactory aggregatorFactory;
+
+    private static final long serialVersionUID = 1L;
+
+    public PreclusteredGroupOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] groupFields,
+            IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory aggregatorFactory,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.groupFields = groupFields;
+        this.comparatorFactories = comparatorFactories;
+        this.aggregatorFactory = aggregatorFactory;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+            throws HyracksDataException {
+        return new PreclusteredGroupOperatorNodePushable(ctx, groupFields, comparatorFactories, aggregatorFactory,
+                recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), recordDescriptors[0]);
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..4acda14
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
@@ -0,0 +1,70 @@
+package edu.uci.ics.hyracks.dataflow.std.group.preclustered;
+
+import java.nio.ByteBuffer;
+
+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.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.IAggregatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+
+class PreclusteredGroupOperatorNodePushable extends
+        AbstractUnaryInputUnaryOutputOperatorNodePushable {
+    private final IHyracksTaskContext ctx;
+    private final int[] groupFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final IAggregatorDescriptorFactory aggregatorFactory;
+    private final RecordDescriptor inRecordDescriptor;
+    private final RecordDescriptor outRecordDescriptor;
+    private PreclusteredGroupWriter pgw;
+
+    PreclusteredGroupOperatorNodePushable(IHyracksTaskContext ctx, int[] groupFields,
+            IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory aggregatorFactory,
+            RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor) {
+        this.ctx = ctx;
+        this.groupFields = groupFields;
+        this.comparatorFactories = comparatorFactories;
+        this.aggregatorFactory = aggregatorFactory;
+        this.inRecordDescriptor = inRecordDescriptor;
+        this.outRecordDescriptor = outRecordDescriptor;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        final IAggregatorDescriptor aggregator = aggregatorFactory.createAggregator(ctx, inRecordDescriptor,
+                outRecordDescriptor, groupFields, groupFields);
+        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, aggregator, inRecordDescriptor,
+                outRecordDescriptor, writer);
+        pgw.open();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        pgw.nextFrame(buffer);
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        pgw.fail();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        pgw.close();
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..d1fec29
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
@@ -0,0 +1,166 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.group.preclustered;
+
+import java.nio.ByteBuffer;
+
+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.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
+import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
+
+public class PreclusteredGroupWriter implements IFrameWriter {
+    private final int[] groupFields;
+    private final IBinaryComparator[] comparators;
+    private final IAggregatorDescriptor aggregator;
+    private final AggregateState aggregateState;
+    private final IFrameWriter writer;
+    private final ByteBuffer copyFrame;
+    private final FrameTupleAccessor inFrameAccessor;
+    private final FrameTupleAccessor copyFrameAccessor;
+
+    private final ByteBuffer outFrame;
+    private final FrameTupleAppender appender;
+    private final ArrayTupleBuilder tupleBuilder;
+
+    private boolean first;
+
+    public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields, IBinaryComparator[] comparators,
+            IAggregatorDescriptor aggregator, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc,
+            IFrameWriter writer) {
+        this.groupFields = groupFields;
+        this.comparators = comparators;
+        this.aggregator = aggregator;
+        this.aggregateState = aggregator.createAggregateStates();
+        this.writer = writer;
+        copyFrame = ctx.allocateFrame();
+        inFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
+        copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
+        copyFrameAccessor.reset(copyFrame);
+
+        outFrame = ctx.allocateFrame();
+        appender = new FrameTupleAppender(ctx.getFrameSize());
+        appender.reset(outFrame, true);
+
+        tupleBuilder = new ArrayTupleBuilder(outRecordDesc.getFields().length);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        writer.open();
+        first = true;
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        inFrameAccessor.reset(buffer);
+        int nTuples = inFrameAccessor.getTupleCount();
+        for (int i = 0; i < nTuples; ++i) {
+            if (first) {
+
+                tupleBuilder.reset();
+                for (int j = 0; j < groupFields.length; j++) {
+                    tupleBuilder.addField(inFrameAccessor, i, groupFields[j]);
+                }
+                aggregator.init(tupleBuilder, inFrameAccessor, i, aggregateState);
+
+                first = false;
+
+            } else {
+                if (i == 0) {
+                    switchGroupIfRequired(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1, inFrameAccessor, i);
+                } else {
+                    switchGroupIfRequired(inFrameAccessor, i - 1, inFrameAccessor, i);
+                }
+
+            }
+        }
+        FrameUtils.copy(buffer, copyFrame);
+    }
+
+    private void switchGroupIfRequired(FrameTupleAccessor prevTupleAccessor, int prevTupleIndex,
+            FrameTupleAccessor currTupleAccessor, int currTupleIndex) throws HyracksDataException {
+        if (!sameGroup(prevTupleAccessor, prevTupleIndex, currTupleAccessor, currTupleIndex)) {
+            writeOutput(prevTupleAccessor, prevTupleIndex);
+
+            tupleBuilder.reset();
+            for (int j = 0; j < groupFields.length; j++) {
+                tupleBuilder.addField(currTupleAccessor, currTupleIndex, groupFields[j]);
+            }
+            aggregator.init(tupleBuilder, currTupleAccessor, currTupleIndex, aggregateState);
+        } else {
+            aggregator.aggregate(currTupleAccessor, currTupleIndex, null, 0, aggregateState);
+        }
+    }
+
+    private void writeOutput(final FrameTupleAccessor lastTupleAccessor, int lastTupleIndex)
+            throws HyracksDataException {
+
+        tupleBuilder.reset();
+        for (int j = 0; j < groupFields.length; j++) {
+            tupleBuilder.addField(lastTupleAccessor, lastTupleIndex, groupFields[j]);
+        }
+        aggregator.outputFinalResult(tupleBuilder, lastTupleAccessor, lastTupleIndex, aggregateState);
+
+        if (!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+                tupleBuilder.getSize())) {
+            FrameUtils.flushFrame(outFrame, writer);
+            appender.reset(outFrame, true);
+            if (!appender.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+                    tupleBuilder.getSize())) {
+                throw new HyracksDataException("The output cannot be fit into a frame.");
+            }
+        }
+
+    }
+
+    private boolean sameGroup(FrameTupleAccessor a1, int t1Idx, FrameTupleAccessor a2, int t2Idx) {
+        for (int i = 0; i < comparators.length; ++i) {
+            int fIdx = groupFields[i];
+            int s1 = a1.getTupleStartOffset(t1Idx) + a1.getFieldSlotsLength() + a1.getFieldStartOffset(t1Idx, fIdx);
+            int l1 = a1.getFieldLength(t1Idx, fIdx);
+            int s2 = a2.getTupleStartOffset(t2Idx) + a2.getFieldSlotsLength() + a2.getFieldStartOffset(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;
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        writer.fail();
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (!first) {
+            writeOutput(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1);
+            if (appender.getTupleCount() > 0) {
+                FrameUtils.flushFrame(outFrame, writer);
+            }
+        }
+        aggregateState.close();
+        writer.close();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
new file mode 100644
index 0000000..6c58d6f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
@@ -0,0 +1,154 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+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.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+
+public class GraceHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final int RPARTITION_ACTIVITY_ID = 0;
+    private static final int SPARTITION_ACTIVITY_ID = 1;
+    private static final int JOIN_ACTIVITY_ID = 2;
+
+    private static final long serialVersionUID = 1L;
+    private final int[] keys0;
+    private final int[] keys1;
+    private final int inputsize0;
+    private final int recordsPerFrame;
+    private final int memsize;
+    private final double factor;
+    private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final boolean isLeftOuter;
+    private final INullWriterFactory[] nullWriterFactories1;
+
+    public GraceHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+            int recordsPerFrame, double factor, int[] keys0, int[] keys1,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 2, 1);
+        this.memsize = memsize;
+        this.inputsize0 = inputsize0;
+        this.recordsPerFrame = recordsPerFrame;
+        this.factor = factor;
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        this.isLeftOuter = false;
+        this.nullWriterFactories1 = null;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    public GraceHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+            int recordsPerFrame, double factor, int[] keys0, int[] keys1,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
+        super(spec, 2, 1);
+        this.memsize = memsize;
+        this.inputsize0 = inputsize0;
+        this.recordsPerFrame = recordsPerFrame;
+        this.factor = factor;
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        this.isLeftOuter = isLeftOuter;
+        this.nullWriterFactories1 = nullWriterFactories1;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        ActivityId rpartAid = new ActivityId(odId, RPARTITION_ACTIVITY_ID);
+        HashPartitionActivityNode rpart = new HashPartitionActivityNode(rpartAid, keys0);
+        ActivityId spartAid = new ActivityId(odId, SPARTITION_ACTIVITY_ID);
+        HashPartitionActivityNode spart = new HashPartitionActivityNode(spartAid, keys1);
+        JoinActivityNode join = new JoinActivityNode(new ActivityId(odId, JOIN_ACTIVITY_ID), rpartAid, spartAid);
+
+        builder.addActivity(this, rpart);
+        builder.addSourceEdge(0, rpart, 0);
+
+        builder.addActivity(this, spart);
+        builder.addSourceEdge(1, spart, 0);
+
+        builder.addActivity(this, join);
+        builder.addBlockingEdge(rpart, spart);
+        builder.addBlockingEdge(spart, join);
+
+        builder.addTargetEdge(0, join, 0);
+    }
+
+    public int getMemorySize() {
+        return memsize;
+    }
+
+    private class HashPartitionActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+        private int keys[];
+
+        public HashPartitionActivityNode(ActivityId id, int keys[]) {
+            super(id);
+            this.keys = keys;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+            return new GraceHashJoinPartitionBuildOperatorNodePushable(ctx, new TaskId(getActivityId(), partition),
+                    keys, hashFunctionFactories, comparatorFactories, (int) Math.ceil(Math.sqrt(inputsize0 * factor
+                            / nPartitions)), recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
+        }
+    }
+
+    private class JoinActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId rpartAid;
+
+        private final ActivityId spartAid;
+
+        public JoinActivityNode(ActivityId id, ActivityId rpartAid, ActivityId spartAid) {
+            super(id);
+            this.rpartAid = rpartAid;
+            this.spartAid = spartAid;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+            final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(rpartAid, 0);
+            final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(spartAid, 0);
+            int numPartitions = (int) Math.ceil(Math.sqrt(inputsize0 * factor / nPartitions));
+
+            return new GraceHashJoinOperatorNodePushable(ctx, new TaskId(new ActivityId(getOperatorId(),
+                    RPARTITION_ACTIVITY_ID), partition), new TaskId(new ActivityId(getOperatorId(),
+                    SPARTITION_ACTIVITY_ID), partition), recordsPerFrame, factor, keys0, keys1, hashFunctionFactories,
+                    comparatorFactories, nullWriterFactories1, rd1, rd0, recordDescriptors[0], numPartitions,
+                    isLeftOuter);
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..91f509d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
@@ -0,0 +1,149 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.nio.ByteBuffer;
+
+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.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+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.FrameTuplePairComparator;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.RepartitionComputerFactory;
+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;
+import edu.uci.ics.hyracks.dataflow.std.structures.ISerializableTable;
+import edu.uci.ics.hyracks.dataflow.std.structures.SerializableHashTable;
+
+class GraceHashJoinOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
+    private final IHyracksTaskContext ctx;
+    private final Object state0Id;
+    private final Object state1Id;
+    private final int[] keys0;
+    private final int[] keys1;
+    private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final INullWriterFactory[] nullWriterFactories;
+    private final RecordDescriptor rd0;
+    private final RecordDescriptor rd1;
+    private final int recordsPerFrame;
+    private final double factor;
+    private final int numPartitions;
+    private final boolean isLeftOuter;
+
+    GraceHashJoinOperatorNodePushable(IHyracksTaskContext ctx, Object state0Id, Object state1Id, int recordsPerFrame,
+            double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFactory[] hashFunctionFactories,
+            IBinaryComparatorFactory[] comparatorFactories, INullWriterFactory[] nullWriterFactories,
+            RecordDescriptor rd1, RecordDescriptor rd0, RecordDescriptor outRecordDescriptor, int numPartitions,
+            boolean isLeftOuter) {
+        this.ctx = ctx;
+        this.state0Id = state0Id;
+        this.state1Id = state1Id;
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        this.nullWriterFactories = nullWriterFactories;
+        this.rd0 = rd0;
+        this.rd1 = rd1;
+        this.numPartitions = numPartitions;
+        this.recordsPerFrame = recordsPerFrame;
+        this.factor = factor;
+        this.isLeftOuter = isLeftOuter;
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+        GraceHashJoinPartitionState rState = (GraceHashJoinPartitionState) ctx.getStateObject(state0Id);
+        GraceHashJoinPartitionState sState = (GraceHashJoinPartitionState) ctx.getStateObject(state1Id);
+        RunFileWriter[] buildWriters = sState.getRunWriters();
+        RunFileWriter[] probeWriters = rState.getRunWriters();
+
+        IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        ITuplePartitionComputer hpcRep0 = new RepartitionComputerFactory(numPartitions,
+                new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)).createPartitioner();
+        ITuplePartitionComputer hpcRep1 = new RepartitionComputerFactory(numPartitions,
+                new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)).createPartitioner();
+
+        final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories.length] : null;
+        if (isLeftOuter) {
+            for (int i = 0; i < nullWriterFactories.length; i++) {
+                nullWriters1[i] = nullWriterFactories[i].createNullWriter();
+            }
+        }
+
+        writer.open();// open for probe
+
+        try {
+
+            ByteBuffer buffer = ctx.allocateFrame();// input
+            // buffer
+            int tableSize = (int) (numPartitions * recordsPerFrame * factor);
+            ISerializableTable table = new SerializableHashTable(tableSize, ctx);
+
+            for (int partitionid = 0; partitionid < numPartitions; partitionid++) {
+                RunFileWriter buildWriter = buildWriters[partitionid];
+                RunFileWriter probeWriter = probeWriters[partitionid];
+                if ((buildWriter == null && !isLeftOuter) || probeWriter == null) {
+                    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);
+
+                // build
+                if (buildWriter != null) {
+                    RunFileReader buildReader = buildWriter.createReader();
+                    buildReader.open();
+                    while (buildReader.nextFrame(buffer)) {
+                        ByteBuffer copyBuffer = ctx.allocateFrame();
+                        FrameUtils.copy(buffer, copyBuffer);
+                        joiner.build(copyBuffer);
+                        buffer.clear();
+                    }
+                    buildReader.close();
+                }
+
+                // probe
+                RunFileReader probeReader = probeWriter.createReader();
+                probeReader.open();
+                while (probeReader.nextFrame(buffer)) {
+                    joiner.join(buffer, writer);
+                    buffer.clear();
+                }
+                probeReader.close();
+                joiner.closeJoin(writer);
+            }
+        } catch (Exception e) {
+            writer.fail();
+            throw new HyracksDataException(e);
+        } finally {
+            writer.close();
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..719b736
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
@@ -0,0 +1,128 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.nio.ByteBuffer;
+
+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.IBinaryHashFunctionFactory;
+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.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.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+
+class GraceHashJoinPartitionBuildOperatorNodePushable extends
+        AbstractUnaryInputSinkOperatorNodePushable {
+    private final IHyracksTaskContext ctx;
+    private final Object stateId;
+    private final int numPartitions;
+    private final IBinaryComparator[] comparators;
+    private final FrameTupleAccessor accessor0;
+    private final ITuplePartitionComputer hpc;
+    private final FrameTupleAppender appender;
+    private ByteBuffer[] outbufs;
+    private GraceHashJoinPartitionState state;
+
+    GraceHashJoinPartitionBuildOperatorNodePushable(IHyracksTaskContext ctx, Object stateId, int[] keys,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            int numPartitions, RecordDescriptor inRecordDescriptor) {
+        this.ctx = ctx;
+        this.stateId = stateId;
+        this.numPartitions = numPartitions;
+        accessor0 = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
+        appender = new FrameTupleAppender(ctx.getFrameSize());
+        hpc = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories).createPartitioner();
+        comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        for (int i = 0; i < numPartitions; i++) {
+            ByteBuffer head = outbufs[i];
+            accessor0.reset(head);
+            if (accessor0.getTupleCount() > 0) {
+                write(i, head);
+            }
+            closeWriter(i);
+        }
+
+        ctx.setStateObject(state);
+    }
+
+    private void closeWriter(int i) throws HyracksDataException {
+        RunFileWriter writer = state.getRunWriters()[i];
+        if (writer != null) {
+            writer.close();
+        }
+    }
+
+    private void write(int i, ByteBuffer head) throws HyracksDataException {
+        RunFileWriter writer = state.getRunWriters()[i];
+        if (writer == null) {
+            FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+                    GraceHashJoinOperatorDescriptor.class.getSimpleName());
+            writer = new RunFileWriter(file, ctx.getIOManager());
+            writer.open();
+            state.getRunWriters()[i] = writer;
+        }
+        writer.nextFrame(head);
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        accessor0.reset(buffer);
+        int tCount = accessor0.getTupleCount();
+        for (int i = 0; i < tCount; ++i) {
+
+            int entry = hpc.partition(accessor0, i, numPartitions);
+            ByteBuffer 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();
+                appender.reset(outbuf, true);
+                if (!appender.append(accessor0, i)) {
+                    throw new HyracksDataException("Item too big to fit in frame");
+                }
+            }
+        }
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        state = new GraceHashJoinPartitionState(ctx.getJobletContext().getJobId(), stateId);
+        outbufs = new ByteBuffer[numPartitions];
+        state.setRunWriters(new RunFileWriter[numPartitions]);
+        for (int i = 0; i < numPartitions; i++) {
+            outbufs[i] = ctx.allocateFrame();
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionState.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionState.java
new file mode 100644
index 0000000..906042e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionState.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
+
+public class GraceHashJoinPartitionState extends AbstractStateObject {
+    private RunFileWriter[] fWriters;
+
+    public GraceHashJoinPartitionState(JobId jobId, Object id) {
+        super(jobId, id);
+    }
+
+    public RunFileWriter[] getRunWriters() {
+        return fWriters;
+    }
+
+    public void setRunWriters(RunFileWriter[] fWriters) {
+        this.fWriters = fWriters;
+    }
+
+    @Override
+    public void toBytes(DataOutput out) throws IOException {
+
+    }
+
+    @Override
+    public void fromBytes(DataInput in) throws IOException {
+
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..4f9b987
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -0,0 +1,558 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+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.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.api.job.JobId;
+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.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.RepartitionComputerFactory;
+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.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.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.structures.ISerializableTable;
+import edu.uci.ics.hyracks.dataflow.std.structures.SerializableHashTable;
+
+public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final int BUILD_AND_PARTITION_ACTIVITY_ID = 0;
+    private static final int PARTITION_AND_JOIN_ACTIVITY_ID = 1;
+
+    private final int memsize;
+    private static final long serialVersionUID = 1L;
+    private final int inputsize0;
+    private final double factor;
+    private final int recordsPerFrame;
+    private final int[] keys0;
+    private final int[] keys1;
+    private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final boolean isLeftOuter;
+    private final INullWriterFactory[] nullWriterFactories1;
+
+    /**
+     * @param spec
+     * @param memsize
+     *            in frames
+     * @param inputsize0
+     *            in frames
+     * @param recordsPerFrame
+     * @param factor
+     * @param keys0
+     * @param keys1
+     * @param hashFunctionFactories
+     * @param comparatorFactories
+     * @param recordDescriptor
+     * @throws HyracksDataException
+     */
+    public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+            int recordsPerFrame, double factor, int[] keys0, int[] keys1,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor) throws HyracksDataException {
+        super(spec, 2, 1);
+        this.memsize = memsize;
+        this.inputsize0 = inputsize0;
+        this.factor = factor;
+        this.recordsPerFrame = recordsPerFrame;
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        this.isLeftOuter = false;
+        this.nullWriterFactories1 = null;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+            int recordsPerFrame, double factor, int[] keys0, int[] keys1,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1)
+            throws HyracksDataException {
+        super(spec, 2, 1);
+        this.memsize = memsize;
+        this.inputsize0 = inputsize0;
+        this.factor = factor;
+        this.recordsPerFrame = recordsPerFrame;
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        this.isLeftOuter = isLeftOuter;
+        this.nullWriterFactories1 = nullWriterFactories1;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        ActivityId p1Aid = new ActivityId(odId, BUILD_AND_PARTITION_ACTIVITY_ID);
+        ActivityId p2Aid = new ActivityId(odId, PARTITION_AND_JOIN_ACTIVITY_ID);
+        BuildAndPartitionActivityNode phase1 = new BuildAndPartitionActivityNode(p1Aid, p2Aid);
+        PartitionAndJoinActivityNode phase2 = new PartitionAndJoinActivityNode(p2Aid, p1Aid);
+
+        builder.addActivity(this, phase1);
+        builder.addSourceEdge(1, phase1, 0);
+
+        builder.addActivity(this, phase2);
+        builder.addSourceEdge(0, phase2, 0);
+
+        builder.addBlockingEdge(phase1, phase2);
+
+        builder.addTargetEdge(0, phase2, 0);
+    }
+
+    public static class BuildAndPartitionTaskState extends AbstractStateObject {
+        private RunFileWriter[] fWriters;
+        private InMemoryHashJoin joiner;
+        private int nPartitions;
+        private int memoryForHashtable;
+
+        public BuildAndPartitionTaskState() {
+        }
+
+        private BuildAndPartitionTaskState(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 BuildAndPartitionActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId joinAid;
+
+        public BuildAndPartitionActivityNode(ActivityId id, ActivityId joinAid) {
+            super(id);
+            this.joinAid = joinAid;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+            final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(joinAid, 0);
+            final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
+            if (isLeftOuter) {
+                for (int i = 0; i < nullWriterFactories1.length; i++) {
+                    nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
+                }
+            }
+
+            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 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();
+
+                @Override
+                public void close() throws HyracksDataException {
+                    if (state.memoryForHashtable != 0)
+                        build(inBuffer);
+
+                    for (int i = 0; i < state.nPartitions; i++) {
+                        ByteBuffer buf = bufferForPartitions[i];
+                        accessorBuild.reset(buf);
+                        if (accessorBuild.getTupleCount() > 0) {
+                            write(i, buf);
+                        }
+                        closeWriter(i);
+                    }
+
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+
+                    if (state.memoryForHashtable != memsize - 2) {
+                        accessorBuild.reset(buffer);
+                        int tCount = accessorBuild.getTupleCount();
+                        for (int i = 0; i < tCount; ++i) {
+                            int entry = -1;
+                            if (state.memoryForHashtable == 0) {
+                                entry = hpcBuild.partition(accessorBuild, i, state.nPartitions);
+                                boolean newBuffer = false;
+                                ByteBuffer bufBi = bufferForPartitions[entry];
+                                while (true) {
+                                    appender.reset(bufBi, newBuffer);
+                                    if (appender.append(accessorBuild, i)) {
+                                        break;
+                                    } else {
+                                        write(entry, bufBi);
+                                        bufBi.clear();
+                                        newBuffer = true;
+                                    }
+                                }
+                            } else {
+                                entry = hpcBuild.partition(accessorBuild, i, (int) (inputsize0 * factor / nPartitions));
+                                if (entry < state.memoryForHashtable) {
+                                    while (true) {
+                                        if (!ftappender.append(accessorBuild, i)) {
+                                            build(inBuffer);
+
+                                            ftappender.reset(inBuffer, true);
+                                        } else {
+                                            break;
+                                        }
+                                    }
+                                } else {
+                                    entry %= state.nPartitions;
+                                    boolean newBuffer = false;
+                                    ByteBuffer bufBi = bufferForPartitions[entry];
+                                    while (true) {
+                                        appender.reset(bufBi, newBuffer);
+                                        if (appender.append(accessorBuild, i)) {
+                                            break;
+                                        } else {
+                                            write(entry, bufBi);
+                                            bufBi.clear();
+                                            newBuffer = true;
+                                        }
+                                    }
+                                }
+                            }
+
+                        }
+                    } else {
+                        build(buffer);
+                    }
+
+                }
+
+                private void build(ByteBuffer inBuffer) throws HyracksDataException {
+                    ByteBuffer copyBuffer = ctx.allocateFrame();
+                    FrameUtils.copy(inBuffer, copyBuffer);
+                    state.joiner.build(copyBuffer);
+                }
+
+                @Override
+                public void open() throws HyracksDataException {
+                    if (memsize > 1) {
+                        if (memsize > inputsize0) {
+                            state.nPartitions = 0;
+                        } else {
+                            state.nPartitions = (int) (Math.ceil((double) (inputsize0 * factor / nPartitions - memsize)
+                                    / (double) (memsize - 1)));
+                        }
+                        if (state.nPartitions <= 0) {
+                            // becomes in-memory HJ
+                            state.memoryForHashtable = memsize - 2;
+                            state.nPartitions = 0;
+                        } else {
+                            state.memoryForHashtable = memsize - state.nPartitions - 2;
+                            if (state.memoryForHashtable < 0) {
+                                state.memoryForHashtable = 0;
+                                state.nPartitions = (int) Math.ceil(Math.sqrt(inputsize0 * factor / nPartitions));
+                            }
+                        }
+                    } else {
+                        throw new HyracksDataException("not enough memory");
+                    }
+
+                    ITuplePartitionComputer hpc0 = new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)
+                            .createPartitioner();
+                    ITuplePartitionComputer hpc1 = new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)
+                            .createPartitioner();
+                    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);
+                    bufferForPartitions = new ByteBuffer[state.nPartitions];
+                    state.fWriters = new RunFileWriter[state.nPartitions];
+                    for (int i = 0; i < state.nPartitions; i++) {
+                        bufferForPartitions[i] = ctx.allocateFrame();
+                    }
+
+                    ftappender.reset(inBuffer, true);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                }
+
+                private void closeWriter(int i) throws HyracksDataException {
+                    RunFileWriter writer = state.fWriters[i];
+                    if (writer != null) {
+                        writer.close();
+                    }
+                }
+
+                private void write(int i, ByteBuffer head) throws HyracksDataException {
+                    RunFileWriter writer = state.fWriters[i];
+                    if (writer == null) {
+                        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+                                BuildAndPartitionActivityNode.class.getSimpleName());
+                        writer = new RunFileWriter(file, ctx.getIOManager());
+                        writer.open();
+                        state.fWriters[i] = writer;
+                    }
+                    writer.nextFrame(head);
+                }
+            };
+            return op;
+        }
+    }
+
+    private class PartitionAndJoinActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId buildAid;
+
+        public PartitionAndJoinActivityNode(ActivityId id, ActivityId buildAid) {
+            super(id);
+            this.buildAid = buildAid;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+            final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
+            if (isLeftOuter) {
+                for (int i = 0; i < nullWriterFactories1.length; i++) {
+                    nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
+                }
+            }
+
+            IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+                private BuildAndPartitionTaskState state;
+                private final FrameTupleAccessor accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), 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 RunFileWriter[] buildWriters;
+                private RunFileWriter[] probeWriters;
+                private ByteBuffer[] bufferForPartitions;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    state = (BuildAndPartitionTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
+                            BUILD_AND_PARTITION_ACTIVITY_ID), partition));
+                    writer.open();
+                    buildWriters = state.fWriters;
+                    probeWriters = new RunFileWriter[state.nPartitions];
+                    bufferForPartitions = new ByteBuffer[state.nPartitions];
+                    for (int i = 0; i < state.nPartitions; i++) {
+                        bufferForPartitions[i] = ctx.allocateFrame();
+                    }
+                    appender.reset(outBuffer, true);
+                    ftap.reset(inBuffer, true);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    if (state.memoryForHashtable != memsize - 2) {
+                        accessorProbe.reset(buffer);
+                        int tupleCount0 = accessorProbe.getTupleCount();
+                        for (int i = 0; i < tupleCount0; ++i) {
+
+                            int entry = -1;
+                            if (state.memoryForHashtable == 0) {
+                                entry = hpcProbe.partition(accessorProbe, i, state.nPartitions);
+                                boolean newBuffer = false;
+                                ByteBuffer outbuf = bufferForPartitions[entry];
+                                while (true) {
+                                    appender.reset(outbuf, newBuffer);
+                                    if (appender.append(accessorProbe, i)) {
+                                        break;
+                                    } else {
+                                        write(entry, outbuf);
+                                        outbuf.clear();
+                                        newBuffer = true;
+                                    }
+                                }
+                            } else {
+                                entry = hpcProbe.partition(accessorProbe, i, (int) (inputsize0 * factor / nPartitions));
+                                if (entry < state.memoryForHashtable) {
+                                    while (true) {
+                                        if (!ftap.append(accessorProbe, i)) {
+                                            state.joiner.join(inBuffer, writer);
+                                            ftap.reset(inBuffer, true);
+                                        } else
+                                            break;
+                                    }
+
+                                } else {
+                                    entry %= state.nPartitions;
+                                    boolean newBuffer = false;
+                                    ByteBuffer outbuf = bufferForPartitions[entry];
+                                    while (true) {
+                                        appender.reset(outbuf, newBuffer);
+                                        if (appender.append(accessorProbe, i)) {
+                                            break;
+                                        } else {
+                                            write(entry, outbuf);
+                                            outbuf.clear();
+                                            newBuffer = true;
+                                        }
+                                    }
+                                }
+                            }
+                        }
+                    } else {
+                        state.joiner.join(buffer, writer);
+                    }
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    state.joiner.join(inBuffer, writer);
+                    state.joiner.closeJoin(writer);
+                    ITuplePartitionComputer hpcRep0 = new RepartitionComputerFactory(state.nPartitions, hpcf0)
+                            .createPartitioner();
+                    ITuplePartitionComputer hpcRep1 = new RepartitionComputerFactory(state.nPartitions, hpcf1)
+                            .createPartitioner();
+                    if (state.memoryForHashtable != memsize - 2) {
+                        for (int i = 0; i < state.nPartitions; i++) {
+                            ByteBuffer buf = bufferForPartitions[i];
+                            accessorProbe.reset(buf);
+                            if (accessorProbe.getTupleCount() > 0) {
+                                write(i, buf);
+                            }
+                            closeWriter(i);
+                        }
+
+                        inBuffer.clear();
+                        int tableSize = -1;
+                        if (state.memoryForHashtable == 0) {
+                            tableSize = (int) (state.nPartitions * recordsPerFrame * factor);
+                        } else {
+                            tableSize = (int) (memsize * recordsPerFrame * factor);
+                        }
+                        for (int partitionid = 0; partitionid < state.nPartitions; partitionid++) {
+                            RunFileWriter buildWriter = buildWriters[partitionid];
+                            RunFileWriter probeWriter = probeWriters[partitionid];
+                            if ((buildWriter == null && !isLeftOuter) || probeWriter == null) {
+                                continue;
+                            }
+                            ISerializableTable table = new SerializableHashTable(tableSize, ctx);
+                            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);
+
+                            if (buildWriter != null) {
+                                RunFileReader buildReader = buildWriter.createReader();
+                                buildReader.open();
+                                while (buildReader.nextFrame(inBuffer)) {
+                                    ByteBuffer copyBuffer = ctx.allocateFrame();
+                                    FrameUtils.copy(inBuffer, copyBuffer);
+                                    joiner.build(copyBuffer);
+                                    inBuffer.clear();
+                                }
+                                buildReader.close();
+                            }
+
+                            // probe
+                            RunFileReader probeReader = probeWriter.createReader();
+                            probeReader.open();
+                            while (probeReader.nextFrame(inBuffer)) {
+                                joiner.join(inBuffer, writer);
+                                inBuffer.clear();
+                            }
+                            probeReader.close();
+                            joiner.closeJoin(writer);
+                        }
+                    }
+                    writer.close();
+                }
+
+                private void closeWriter(int i) throws HyracksDataException {
+                    RunFileWriter writer = probeWriters[i];
+                    if (writer != null) {
+                        writer.close();
+                    }
+                }
+
+                private void write(int i, ByteBuffer head) throws HyracksDataException {
+                    RunFileWriter writer = probeWriters[i];
+                    if (writer == null) {
+                        FileReference file = ctx.createManagedWorkspaceFile(PartitionAndJoinActivityNode.class
+                                .getSimpleName());
+                        writer = new RunFileWriter(file, ctx.getIOManager());
+                        writer.open();
+                        probeWriters[i] = writer;
+                    }
+                    writer.nextFrame(head);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    writer.fail();
+                }
+            };
+            return op;
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..3e5e30f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -0,0 +1,167 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.io.DataOutput;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+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.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+import edu.uci.ics.hyracks.dataflow.std.structures.ISerializableTable;
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public class InMemoryHashJoin {
+	
+    private final List<ByteBuffer> buffers;
+    private final FrameTupleAccessor accessorBuild;
+    private final ITuplePartitionComputer tpcBuild;
+    private final FrameTupleAccessor accessorProbe;
+    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;
+	private final int tableSize;
+    private final TuplePointer storedTuplePointer;
+    
+    public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
+            ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
+            FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1, ISerializableTable table)
+            throws HyracksDataException {
+    	this.tableSize = tableSize;
+       	this.table = table;
+       	storedTuplePointer = new TuplePointer();
+       	buffers = new ArrayList<ByteBuffer>();
+        this.accessorBuild = accessor1;
+        this.tpcBuild = tpc1;
+        this.accessorProbe = accessor0;
+        this.tpcProbe = tpc0;
+        appender = new FrameTupleAppender(ctx.getFrameSize());
+        tpComparator = comparator;
+        outBuffer = ctx.allocateFrame();
+        appender.reset(outBuffer, true);
+        this.isLeftOuter = isLeftOuter;
+        if (isLeftOuter) {
+            int fieldCountOuter = accessor1.getFieldCount();
+            nullTupleBuild = new ArrayTupleBuilder(fieldCountOuter);
+            DataOutput out = nullTupleBuild.getDataOutput();
+            for (int i = 0; i < fieldCountOuter; i++) {
+                nullWriters1[i].writeNull(out);
+                nullTupleBuild.addFieldEndOffset();
+            }
+        } else {
+            nullTupleBuild = null;
+        }
+    }
+
+    public void build(ByteBuffer buffer) throws HyracksDataException {
+        buffers.add(buffer);
+        int bIndex = buffers.size() - 1;
+        accessorBuild.reset(buffer);
+        int tCount = accessorBuild.getTupleCount();
+        for (int i = 0; i < tCount; ++i) {
+            int entry = tpcBuild.partition(accessorBuild, i, tableSize);
+            storedTuplePointer.frameIndex = bIndex;
+            storedTuplePointer.tupleIndex = i;
+            table.insert(entry, storedTuplePointer);
+        }
+    }
+
+    public void join(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
+        accessorProbe.reset(buffer);
+        int tupleCount0 = accessorProbe.getTupleCount();
+        for (int i = 0; i < tupleCount0; ++i) {
+            int entry = tpcProbe.partition(accessorProbe, i, tableSize);
+            boolean matchFound = false;
+            int offset = 0;
+            do {
+                table.getTuplePointer(entry, offset++, storedTuplePointer);
+                if (storedTuplePointer.frameIndex < 0)
+                    break;
+                int bIndex = storedTuplePointer.frameIndex;
+                int tIndex = storedTuplePointer.tupleIndex;
+                accessorBuild.reset(buffers.get(bIndex));
+                int c = tpComparator.compare(accessorProbe, i, accessorBuild, tIndex);
+                if (c == 0) {
+                    matchFound = true;
+                    if (!appender.appendConcat(accessorProbe, i, accessorBuild, tIndex)) {
+                        flushFrame(outBuffer, writer);
+                        appender.reset(outBuffer, true);
+                        if (!appender.appendConcat(accessorProbe, i, accessorBuild, tIndex)) {
+                            throw new IllegalStateException();
+                        }
+                    }
+                }
+            } 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 IllegalStateException();
+                    }
+                }
+            }
+        }
+    }
+
+    public void closeJoin(IFrameWriter writer) throws HyracksDataException {
+        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());
+    }
+
+    private static class Link {
+        private static final int INIT_POINTERS_SIZE = 8;
+
+        long[] pointers;
+        int size;
+
+        Link() {
+            pointers = new long[INIT_POINTERS_SIZE];
+            size = 0;
+        }
+
+        void add(long pointer) {
+            if (size >= pointers.length) {
+                pointers = Arrays.copyOf(pointers, pointers.length * 2);
+            }
+            pointers[size++] = pointer;
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..e0a5613
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -0,0 +1,231 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+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.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+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.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+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.common.data.partition.FieldHashPartitionComputerFactory;
+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.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.structures.ISerializableTable;
+import edu.uci.ics.hyracks.dataflow.std.structures.SerializableHashTable;
+
+public class InMemoryHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private final int[] keys0;
+    private final int[] keys1;
+    private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final boolean isLeftOuter;
+    private final INullWriterFactory[] nullWriterFactories1;
+    private final int tableSize;
+
+    public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, int tableSize) {
+        super(spec, 2, 1);
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        recordDescriptors[0] = recordDescriptor;
+        this.isLeftOuter = false;
+        this.nullWriterFactories1 = null;
+        this.tableSize = tableSize;
+    }
+
+    public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
+            IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1,
+            int tableSize) {
+        super(spec, 2, 1);
+        this.keys0 = keys0;
+        this.keys1 = keys1;
+        this.hashFunctionFactories = hashFunctionFactories;
+        this.comparatorFactories = comparatorFactories;
+        recordDescriptors[0] = recordDescriptor;
+        this.isLeftOuter = isLeftOuter;
+        this.nullWriterFactories1 = nullWriterFactories1;
+        this.tableSize = tableSize;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        ActivityId hbaId = new ActivityId(odId, 0);
+        ActivityId hpaId = new ActivityId(odId, 1);
+        HashBuildActivityNode hba = new HashBuildActivityNode(hbaId, hpaId);
+        HashProbeActivityNode hpa = new HashProbeActivityNode(hpaId);
+
+        builder.addActivity(this, hba);
+        builder.addSourceEdge(1, hba, 0);
+
+        builder.addActivity(this, hpa);
+        builder.addSourceEdge(0, hpa, 0);
+
+        builder.addTargetEdge(0, hpa, 0);
+
+        builder.addBlockingEdge(hba, hpa);
+    }
+
+    public static class HashBuildTaskState extends AbstractStateObject {
+        private InMemoryHashJoin joiner;
+
+        public HashBuildTaskState() {
+        }
+
+        private HashBuildTaskState(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 HashBuildActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId hpaId;
+
+        public HashBuildActivityNode(ActivityId id, ActivityId hpaId) {
+            super(id);
+            this.hpaId = hpaId;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(hpaId, 0);
+            final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; ++i) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
+            if (isLeftOuter) {
+                for (int i = 0; i < nullWriterFactories1.length; i++) {
+                    nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
+                }
+            }
+
+            IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
+                private HashBuildTaskState state;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    ITuplePartitionComputer hpc0 = new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)
+                            .createPartitioner();
+                    ITuplePartitionComputer hpc1 = new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)
+                            .createPartitioner();
+                    state = new HashBuildTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(),
+                            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,
+                                    comparators), isLeftOuter, nullWriters1, table);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    ByteBuffer copyBuffer = ctx.allocateFrame();
+                    FrameUtils.copy(buffer, copyBuffer);
+                    state.joiner.build(copyBuffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                }
+            };
+            return op;
+        }
+    }
+
+    private class HashProbeActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public HashProbeActivityNode(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+                private HashBuildTaskState state;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    state = (HashBuildTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(), 0),
+                            partition));
+                    writer.open();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    state.joiner.join(buffer, writer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    state.joiner.closeJoin(writer);
+                    writer.close();
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    writer.fail();
+                }
+            };
+            return op;
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/JoinComparator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/JoinComparator.java
new file mode 100644
index 0000000..4891be8
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/JoinComparator.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
+
+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) {
+         int tStart0 = accessor0.getTupleStartOffset(tIndex0);
+         int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
+
+         int tStart1 = accessor1.getTupleStartOffset(tIndex1);
+         int fStartOffset1 = accessor1.getFieldSlotsLength() + tStart1;
+
+         int fStart0 = accessor0.getFieldStartOffset(tIndex0, field0);
+         int fEnd0 = accessor0.getFieldEndOffset(tIndex0, field0);
+         int fLen0 = fEnd0 - fStart0;
+
+         int fStart1 = accessor1.getFieldStartOffset(tIndex1, field1);
+         int fEnd1 = accessor1.getFieldEndOffset(tIndex1, field1);
+         int fLen1 = fEnd1 - fStart1;
+
+         int c = bComparator.compare(accessor0.getBuffer().array(), fStart0 + fStartOffset0, fLen0, accessor1
+                 .getBuffer().array(), fStart1 + fStartOffset1, fLen1);
+         if (c != 0) {
+             return c;
+         }
+         return 0;
+     }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/JoinComparatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/JoinComparatorFactory.java
new file mode 100644
index 0000000..377e9e0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/JoinComparatorFactory.java
@@ -0,0 +1,26 @@
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+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.ITuplePairComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
+
+public 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);
+    }
+
+}
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
new file mode 100644
index 0000000..7e84229
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -0,0 +1,166 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+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.ITuplePairComparator;
+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;
+
+public class NestedLoopJoin {
+    private final FrameTupleAccessor accessorInner;
+    private final FrameTupleAccessor accessorOuter;
+    private final FrameTupleAppender appender;
+    private final ITuplePairComparator tpComparator;
+    private final ByteBuffer outBuffer;
+    private final ByteBuffer innerBuffer;
+    private final List<ByteBuffer> outBuffers;
+    private final int memSize;
+    private final IHyracksTaskContext ctx;
+    private RunFileReader runFileReader;
+    private int currentMemSize = 0;
+    private final RunFileWriter runFileWriter;
+
+    public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
+            ITuplePairComparator comparators, int memSize) throws HyracksDataException {
+        this.accessorInner = accessor1;
+        this.accessorOuter = accessor0;
+        this.appender = new FrameTupleAppender(ctx.getFrameSize());
+        this.tpComparator = comparators;
+        this.outBuffer = ctx.allocateFrame();
+        this.innerBuffer = ctx.allocateFrame();
+        this.appender.reset(outBuffer, true);
+        this.outBuffers = new ArrayList<ByteBuffer>();
+        this.memSize = memSize;
+        this.ctx = ctx;
+
+        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+                this.getClass().getSimpleName() + this.toString());
+        runFileWriter = new RunFileWriter(file, ctx.getIOManager());
+        runFileWriter.open();
+    }
+
+    public void cache(ByteBuffer buffer) throws HyracksDataException {
+        runFileWriter.nextFrame(buffer);
+    }
+
+    public void join(ByteBuffer outerBuffer, IFrameWriter writer) throws HyracksDataException {
+        if (outBuffers.size() < memSize - 3) {
+            createAndCopyFrame(outerBuffer);
+            return;
+        }
+        if (currentMemSize < memSize - 3) {
+            reloadFrame(outerBuffer);
+            return;
+        }
+        for (ByteBuffer outBuffer : outBuffers) {
+            runFileReader = runFileWriter.createReader();
+            runFileReader.open();
+            while (runFileReader.nextFrame(innerBuffer)) {
+                blockJoin(outBuffer, innerBuffer, writer);
+            }
+            runFileReader.close();
+        }
+        currentMemSize = 0;
+        reloadFrame(outerBuffer);
+    }
+
+    private void createAndCopyFrame(ByteBuffer outerBuffer) {
+        ByteBuffer outerBufferCopy = ctx.allocateFrame();
+        FrameUtils.copy(outerBuffer, outerBufferCopy);
+        outBuffers.add(outerBufferCopy);
+        currentMemSize++;
+    }
+
+    private void reloadFrame(ByteBuffer outerBuffer) {
+        outBuffers.get(currentMemSize).clear();
+        FrameUtils.copy(outerBuffer, outBuffers.get(currentMemSize));
+        currentMemSize++;
+    }
+
+    private void blockJoin(ByteBuffer outerBuffer, ByteBuffer innerBuffer, IFrameWriter writer)
+            throws HyracksDataException {
+        accessorOuter.reset(outerBuffer);
+        accessorInner.reset(innerBuffer);
+        int tupleCount0 = accessorOuter.getTupleCount();
+        int tupleCount1 = accessorInner.getTupleCount();
+
+        for (int i = 0; i < tupleCount0; ++i) {
+            for (int j = 0; j < tupleCount1; ++j) {
+                int c = compare(accessorOuter, i, accessorInner, j);
+                if (c == 0) {
+                    if (!appender.appendConcat(accessorOuter, i, accessorInner, j)) {
+                        flushFrame(outBuffer, writer);
+                        appender.reset(outBuffer, true);
+                        if (!appender.appendConcat(accessorOuter, i, accessorInner, j)) {
+                            throw new IllegalStateException();
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    public void closeCache() throws HyracksDataException {
+        if (runFileWriter != null) {
+            runFileWriter.close();
+        }
+    }
+
+    public void closeJoin(IFrameWriter writer) throws HyracksDataException {
+        for (int i = 0; i < currentMemSize; i++) {
+            ByteBuffer outBuffer = outBuffers.get(i);
+            runFileReader = runFileWriter.createReader();
+            runFileReader.open();
+            while (runFileReader.nextFrame(innerBuffer)) {
+                blockJoin(outBuffer, innerBuffer, 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());
+    }
+
+    private int compare(FrameTupleAccessor accessor0, int tIndex0, FrameTupleAccessor accessor1, int tIndex1)
+            throws HyracksDataException {
+        int c = tpComparator.compare(accessor0, tIndex0, accessor1, tIndex1);
+        if (c != 0) {
+            return c;
+        }
+        return 0;
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..a699703
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -0,0 +1,187 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+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.IRecordDescriptorProvider;
+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.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.api.job.JobId;
+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.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.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class NestedLoopJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final int JOIN_CACHE_ACTIVITY_ID = 0;
+    private static final int NL_JOIN_ACTIVITY_ID = 1;
+
+    private static final long serialVersionUID = 1L;
+    private final ITuplePairComparatorFactory comparatorFactory;
+    private final int memSize;
+
+    public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
+            ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize) {
+        super(spec, 2, 1);
+        this.comparatorFactory = comparatorFactory;
+        this.recordDescriptors[0] = recordDescriptor;
+        this.memSize = memSize;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        ActivityId jcaId = new ActivityId(getOperatorId(), JOIN_CACHE_ACTIVITY_ID);
+        ActivityId nljAid = new ActivityId(getOperatorId(), NL_JOIN_ACTIVITY_ID);
+        JoinCacheActivityNode jc = new JoinCacheActivityNode(jcaId, nljAid);
+        NestedLoopJoinActivityNode nlj = new NestedLoopJoinActivityNode(nljAid);
+
+        builder.addActivity(this, jc);
+        builder.addSourceEdge(1, jc, 0);
+
+        builder.addActivity(this, nlj);
+        builder.addSourceEdge(0, nlj, 0);
+
+        builder.addTargetEdge(0, nlj, 0);
+        builder.addBlockingEdge(jc, nlj);
+    }
+
+    public static class JoinCacheTaskState extends AbstractStateObject {
+        private NestedLoopJoin joiner;
+
+        public JoinCacheTaskState() {
+        }
+
+        private JoinCacheTaskState(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 JoinCacheActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId nljAid;
+
+        public JoinCacheActivityNode(ActivityId id, ActivityId nljAid) {
+            super(id);
+            this.nljAid = nljAid;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
+            final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator(ctx);
+
+            IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
+                private JoinCacheTaskState state;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    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);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    ByteBuffer copyBuffer = ctx.allocateFrame();
+                    FrameUtils.copy(buffer, copyBuffer);
+                    FrameUtils.makeReadable(copyBuffer);
+                    state.joiner.cache(copyBuffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    state.joiner.closeCache();
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                }
+            };
+            return op;
+        }
+    }
+
+    private class NestedLoopJoinActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public NestedLoopJoinActivityNode(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+
+            IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+                private JoinCacheTaskState state;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    state = (JoinCacheTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
+                            JOIN_CACHE_ACTIVITY_ID), partition));
+                    writer.open();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    state.joiner.join(buffer, writer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    state.joiner.closeJoin(writer);
+                    writer.close();
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    writer.fail();
+                }
+            };
+            return op;
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..2905574
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -0,0 +1,607 @@
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.BitSet;
+
+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.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+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.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.io.FrameTuplePairComparator;
+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.structures.ISerializableTable;
+import edu.uci.ics.hyracks.dataflow.std.structures.SerializableHashTable;
+
+/**
+ * @author pouria
+       This class mainly applies one level of HHJ on a pair of
+       relations. It is always called by the descriptor.
+ */
+public class OptimizedHybridHashJoin {
+
+    private final int NO_MORE_FREE_BUFFER = -1;
+    private final int END_OF_PARTITION = -1;
+    private final int INVALID_BUFFER = -2;
+    private final int UNALLOCATED_FRAME = -3;
+    private final int BUFFER_FOR_RESIDENT_PARTS = -1;
+    
+    private IHyracksTaskContext ctx;
+
+    private final String rel0Name;
+    private final String rel1Name;
+
+    private final int[] buildKeys;
+    private final int[] probeKeys;
+
+    private final IBinaryComparator[] comparators;
+
+    private ITuplePartitionComputer buildHpc;
+    private ITuplePartitionComputer probeHpc;
+
+    private final RecordDescriptor buildRd;
+    private final RecordDescriptor probeRd;
+
+    private RunFileWriter[] buildRFWriters; //writing spilled build partitions
+    private RunFileWriter[] probeRFWriters; //writing spilled probe partitions
+
+    private final boolean isLeftOuter;
+    private final INullWriter[] nullWriters1;
+
+    private ByteBuffer[] memBuffs; //Memory buffers for build
+    private int[] curPBuff; //Current (last) Buffer for each partition
+    private int[] nextBuff; //Next buffer in the partition's buffer chain
+    private int[] buildPSizeInTups; //Size of build partitions (in tuples)
+    private int[] probePSizeInTups; //Size of probe partitions (in tuples)
+    private int nextFreeBuffIx; //Index of next available free buffer to allocate/use
+    private BitSet pStatus; //0=resident, 1=spilled
+    private int numOfPartitions;
+    private int memForJoin;
+    private InMemoryHashJoin inMemJoiner; //Used for joining resident partitions
+
+    private final FrameTupleAccessor accessorBuild;
+    private final FrameTupleAccessor accessorProbe;
+    private FrameTupleAppender buildTupAppender;
+    private FrameTupleAppender probeTupAppenderToResident;
+    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 int[] buildPSizeInFrames; //Used for partition tuning
+    private int freeFramesCounter; //Used for partition tuning
+
+    public OptimizedHybridHashJoin(IHyracksTaskContext ctx, int memForJoin, int numOfPartitions, String rel0Name,
+            String rel1Name, int[] keys0, int[] keys1, IBinaryComparator[] comparators, RecordDescriptor buildRd,
+            RecordDescriptor probeRd, ITuplePartitionComputer probeHpc, ITuplePartitionComputer buildHpc) {
+        this.ctx = ctx;
+        this.memForJoin = memForJoin;
+        this.buildRd = buildRd;
+        this.probeRd = probeRd;
+        this.buildHpc = probeHpc;
+        this.probeHpc = buildHpc;
+        this.buildKeys = keys0;
+        this.probeKeys = keys1;
+        this.comparators = comparators;
+        this.rel0Name = rel0Name;
+        this.rel1Name = rel1Name;
+
+        this.numOfPartitions = numOfPartitions;
+        this.buildRFWriters = new RunFileWriter[numOfPartitions];
+        this.probeRFWriters = new RunFileWriter[numOfPartitions];
+
+        this.accessorBuild = new FrameTupleAccessor(ctx.getFrameSize(), buildRd);
+        this.accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), probeRd);
+
+        this.isLeftOuter = false;
+        this.nullWriters1 = null;
+
+    }
+
+    public OptimizedHybridHashJoin(IHyracksTaskContext ctx, int memForJoin, int numOfPartitions, String rel0Name,
+            String rel1Name, int[] keys0, int[] keys1, IBinaryComparator[] comparators, RecordDescriptor buildRd,
+            RecordDescriptor probeRd, ITuplePartitionComputer probeHpc, ITuplePartitionComputer buildHpc,
+            boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
+        this.ctx = ctx;
+        this.memForJoin = memForJoin;
+        this.buildRd = buildRd;
+        this.probeRd = probeRd;
+        this.buildHpc = probeHpc;
+        this.probeHpc = buildHpc;
+        this.buildKeys = keys0;
+        this.probeKeys = keys1;
+        this.comparators = comparators;
+        this.rel0Name = rel0Name;
+        this.rel1Name = rel1Name;
+
+        this.numOfPartitions = numOfPartitions;
+        this.buildRFWriters = new RunFileWriter[numOfPartitions];
+        this.probeRFWriters = new RunFileWriter[numOfPartitions];
+
+        this.accessorBuild = new FrameTupleAccessor(ctx.getFrameSize(), buildRd);
+        this.accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), probeRd);
+
+        this.isLeftOuter = isLeftOuter;
+
+        this.nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
+        if (isLeftOuter) {
+            for (int i = 0; i < nullWriterFactories1.length; i++) {
+                nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
+            }
+        }
+    }
+
+    public void initBuild() {
+        memBuffs = new ByteBuffer[memForJoin];
+        curPBuff = new int[numOfPartitions];
+        nextBuff = new int[memForJoin];
+        pStatus = new BitSet(numOfPartitions);
+        buildPSizeInTups = new int[numOfPartitions];
+
+        buildPSizeInFrames = new int[numOfPartitions];
+        freeFramesCounter = memForJoin - numOfPartitions;
+
+        for (int i = 0; i < numOfPartitions; i++) { //Allocating one buffer per partition and setting as the head of the chain of buffers for that partition
+            memBuffs[i] = ctx.allocateFrame();
+            curPBuff[i] = i;
+            nextBuff[i] = -1;
+            buildPSizeInFrames[i] = 1; //The dedicated initial buffer
+        }
+
+        nextFreeBuffIx = ((numOfPartitions < memForJoin) ? numOfPartitions : NO_MORE_FREE_BUFFER); //Setting the chain of unallocated frames
+        for (int i = numOfPartitions; i < memBuffs.length; i++) {
+            nextBuff[i] = UNALLOCATED_FRAME;
+        }
+
+        buildTupAppender = new FrameTupleAppender(ctx.getFrameSize());
+
+    }
+
+    public void build(ByteBuffer buffer) throws HyracksDataException {
+        accessorBuild.reset(buffer);
+        int tupleCount = accessorBuild.getTupleCount();
+        for (int i = 0; i < tupleCount; ++i) {
+            int pid = buildHpc.partition(accessorBuild, i, numOfPartitions);
+            processTuple(i, pid);
+            buildPSizeInTups[pid]++;
+        }
+
+    }
+
+    private void processTuple(int tid, int pid) throws HyracksDataException {
+        ByteBuffer partition = memBuffs[curPBuff[pid]]; //Getting current buffer for the target partition
+
+        if (!pStatus.get(pid)) { //resident partition
+            buildTupAppender.reset(partition, false);
+            while (true) {
+                if (buildTupAppender.append(accessorBuild, tid)) { //Tuple added to resident partition successfully
+                    break;
+                }
+                //partition does not have enough room
+                int newBuffIx = allocateFreeBuffer(pid);
+                if (newBuffIx == NO_MORE_FREE_BUFFER) { //Spill one partition
+                    int pidToSpill = selectPartitionToSpill();
+                    if (pidToSpill == -1) { //No more partition to spill
+                        throw new HyracksDataException("not enough memory for Hash Join (Allocation exceeds the limit)");
+                    }
+                    spillPartition(pidToSpill);
+                    buildTupAppender.reset(memBuffs[pidToSpill], true);
+                    processTuple(tid, pid);
+                    break;
+                }  //New Buffer allocated successfully
+                partition = memBuffs[curPBuff[pid]]; //Current Buffer for the partition is now updated by allocateFreeBuffer() call above
+                buildTupAppender.reset(partition, true);
+                if (!buildTupAppender.append(accessorBuild, tid)) {
+                    throw new HyracksDataException("Invalid State (Can not append to newly allocated buffer)");
+                }
+                buildPSizeInFrames[pid]++;
+                break;
+            }
+        } else { //spilled partition
+            boolean needClear = false;
+            while (true) {
+                buildTupAppender.reset(partition, needClear);
+                if (buildTupAppender.append(accessorBuild, tid)) {
+                    break;
+                }
+                //Dedicated in-memory buffer for the partition is full, needed to be flushed first 
+                buildWrite(pid, partition);
+                partition.clear();
+                needClear = true;
+                buildPSizeInFrames[pid]++;
+            }
+        }
+    }
+
+    private int allocateFreeBuffer(int pid) {
+        if (nextFreeBuffIx != NO_MORE_FREE_BUFFER) {
+            if (memBuffs[nextFreeBuffIx] == null) {
+                memBuffs[nextFreeBuffIx] = ctx.allocateFrame();
+            }
+            int curPartBuffIx = curPBuff[pid];
+            curPBuff[pid] = nextFreeBuffIx;
+            int oldNext = nextBuff[nextFreeBuffIx];
+            nextBuff[nextFreeBuffIx] = curPartBuffIx;
+            if (oldNext == UNALLOCATED_FRAME) {
+                nextFreeBuffIx++;
+                if (nextFreeBuffIx == memForJoin) { //No more free buffer
+                    nextFreeBuffIx = NO_MORE_FREE_BUFFER;
+                }
+            } else {
+                nextFreeBuffIx = oldNext;
+            }
+            (memBuffs[curPBuff[pid]]).clear();
+
+            freeFramesCounter--;
+            return (curPBuff[pid]);
+        } else {
+            return NO_MORE_FREE_BUFFER; //A partitions needs to be spilled (if feasible)
+        }
+    }
+
+    private int selectPartitionToSpill() {
+        int maxSize = -1;
+        int partitionToSpill = -1;
+        for (int i = 0; i < buildPSizeInTups.length; i++) { //Find the largest partition, to spill
+            if (!pStatus.get(i) && (buildPSizeInTups[i] > maxSize)) {
+                maxSize = buildPSizeInTups[i];
+                partitionToSpill = i;
+            }
+        }
+        return partitionToSpill;
+    }
+
+    private void spillPartition(int pid) throws HyracksDataException {
+        int curBuffIx = curPBuff[pid];
+        ByteBuffer buff = null;
+        while (curBuffIx != END_OF_PARTITION) {
+            buff = memBuffs[curBuffIx];
+            buildWrite(pid, buff);
+            buff.clear();
+
+            int freedBuffIx = curBuffIx;
+            curBuffIx = nextBuff[curBuffIx];
+
+            if (freedBuffIx != pid) {
+                nextBuff[freedBuffIx] = nextFreeBuffIx;
+                nextFreeBuffIx = freedBuffIx;
+                freeFramesCounter++;
+            }
+        }
+        curPBuff[pid] = pid;
+        pStatus.set(pid);
+    }
+
+    private void buildWrite(int pid, ByteBuffer buff) throws HyracksDataException {
+        RunFileWriter writer = buildRFWriters[pid];
+        if (writer == null) {
+            FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(rel0Name);
+            writer = new RunFileWriter(file, ctx.getIOManager());
+            writer.open();
+            buildRFWriters[pid] = writer;
+        }
+        writer.nextFrame(buff);
+    }
+
+    public void closeBuild() throws HyracksDataException {
+        for (int i = 0; i < numOfPartitions; i++) { //Remove Empty Partitions' allocated frame
+            if (buildPSizeInTups[i] == 0) {
+                buildPSizeInFrames[i]--;
+                nextBuff[curPBuff[i]] = nextFreeBuffIx;
+                nextFreeBuffIx = curPBuff[i];
+                curPBuff[i] = INVALID_BUFFER;
+                freeFramesCounter++;
+            }
+        }
+
+        ByteBuffer buff = null;
+        for (int i = pStatus.nextSetBit(0); i >= 0; i = pStatus.nextSetBit(i + 1)) { //flushing and DeAllocating the dedicated buffers for the spilled partitions
+            buff = memBuffs[i];
+            accessorBuild.reset(buff);
+            if (accessorBuild.getTupleCount() > 0) {
+                buildWrite(i, buff);
+                buildPSizeInFrames[i]++;
+            }
+            nextBuff[i] = nextFreeBuffIx;
+            nextFreeBuffIx = i;
+            freeFramesCounter++;
+            curPBuff[i] = INVALID_BUFFER;
+
+            if (buildRFWriters[i] != null) {
+                buildRFWriters[i].close();
+            }
+        }
+
+        partitionTune(); //Trying to bring back as many spilled partitions as possible, making them resident
+
+        int inMemTupCount = 0;
+        numOfSpilledParts = 0;
+
+        for (int i = 0; i < numOfPartitions; i++) {
+            if (!pStatus.get(i)) {
+                inMemTupCount += buildPSizeInTups[i];
+            } else {
+                numOfSpilledParts++;
+            }
+        }
+
+        createInMemoryJoiner(inMemTupCount);
+        cacheInMemJoin();
+    }
+
+    private void partitionTune() throws HyracksDataException {
+        reloadBuffer = ctx.allocateFrame();
+        ArrayList<Integer> reloadSet = selectPartitionsToReload();
+        for (int i = 0; i < reloadSet.size(); i++) {
+            int pid = reloadSet.get(i);
+            int[] buffsToLoad = new int[buildPSizeInFrames[pid]];
+            for (int j = 0; j < buffsToLoad.length; j++) {
+                buffsToLoad[j] = nextFreeBuffIx;
+                int oldNext = nextBuff[nextFreeBuffIx];
+                if (oldNext == UNALLOCATED_FRAME) {
+                    nextFreeBuffIx++;
+                    if (nextFreeBuffIx == memForJoin) { //No more free buffer
+                        nextFreeBuffIx = NO_MORE_FREE_BUFFER;
+                    }
+                } else {
+                    nextFreeBuffIx = oldNext;
+                }
+
+            }
+            curPBuff[pid] = buffsToLoad[0];
+            for (int k = 1; k < buffsToLoad.length; k++) {
+                nextBuff[buffsToLoad[k - 1]] = buffsToLoad[k];
+            }
+            loadPartitionInMem(pid, buildRFWriters[pid], buffsToLoad);
+        }
+        reloadSet.clear();
+        reloadSet = null;
+    }
+
+    private void loadPartitionInMem(int pid, RunFileWriter wr, int[] buffs) throws HyracksDataException {
+        RunFileReader r = wr.createReader();
+        r.open();
+        int counter = 0;
+        ByteBuffer mBuff = null;
+        reloadBuffer.clear();
+        while (r.nextFrame(reloadBuffer)) {
+            mBuff = memBuffs[buffs[counter]];
+            if (mBuff == null) {
+                mBuff = ctx.allocateFrame();
+                memBuffs[buffs[counter]] = mBuff;
+            }
+            FrameUtils.copy(reloadBuffer, mBuff);
+            counter++;
+            reloadBuffer.clear();
+        }
+
+        int curNext = nextBuff[buffs[buffs.length - 1]];
+        nextBuff[buffs[buffs.length - 1]] = END_OF_PARTITION;
+        nextFreeBuffIx = curNext;
+
+        r.close();
+        pStatus.set(pid, false);
+        buildRFWriters[pid] = null;
+    }
+
+    private ArrayList<Integer> selectPartitionsToReload() {
+        ArrayList<Integer> p = new ArrayList<Integer>();
+        for (int i = pStatus.nextSetBit(0); i >= 0; i = pStatus.nextSetBit(i + 1)) {
+            if (buildPSizeInFrames[i]>0 && (freeFramesCounter - buildPSizeInFrames[i] >= 0) ) {
+                p.add(i);
+                freeFramesCounter -= buildPSizeInFrames[i];
+            }
+            if (freeFramesCounter < 1) { //No more free buffer available
+                return p;
+            }
+        }
+        return p;
+    }
+
+    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);
+    }
+
+    private void cacheInMemJoin() throws HyracksDataException {
+
+        for (int pid = 0; pid < numOfPartitions; pid++) {
+            if (!pStatus.get(pid)) {
+                int nextBuffIx = curPBuff[pid];
+                while (nextBuffIx > -1) { //It is not Invalid or End_Of_Partition
+                    inMemJoiner.build(memBuffs[nextBuffIx]);
+                    nextBuffIx = nextBuff[nextBuffIx];
+                }
+            }
+        }
+    }
+
+    public void initProbe() {
+
+        sPartBuffs = new ByteBuffer[numOfSpilledParts];
+        for (int i = 0; i < numOfSpilledParts; i++) {
+            sPartBuffs[i] = ctx.allocateFrame();
+        }
+        curPBuff = new int[numOfPartitions];
+        int nextBuffIxToAlloc = 0;
+        /* We only need to allocate one frame per spilled partition. 
+         * Resident partitions do not need frames in probe, as their tuples join 
+         * immediately with the resident build tuples using the inMemoryHashJoin */
+        for (int i = 0; i < numOfPartitions; i++) { 
+            curPBuff[i] = (pStatus.get(i)) ? nextBuffIxToAlloc++ : BUFFER_FOR_RESIDENT_PARTS;
+        }
+        probePSizeInTups = new int[numOfPartitions];
+        probeRFWriters = new RunFileWriter[numOfPartitions];
+
+        probeResBuff = ctx.allocateFrame();
+
+        probeTupAppenderToResident = new FrameTupleAppender(ctx.getFrameSize());
+        probeTupAppenderToResident.reset(probeResBuff, true);
+
+        probeTupAppenderToSpilled = new FrameTupleAppender(ctx.getFrameSize());
+
+    }
+
+    public void probe(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
+
+        accessorProbe.reset(buffer);
+        int tupleCount = accessorProbe.getTupleCount();
+
+        if (numOfSpilledParts == 0) {
+            inMemJoiner.join(buffer, writer);
+            return;
+        }
+
+        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 (pStatus.get(pid)) { //pid is Spilled
+                    boolean needToClear = false;
+                    ByteBuffer buff = sPartBuffs[curPBuff[pid]];
+                    while (true) {
+                        probeTupAppenderToSpilled.reset(buff, needToClear);
+                        if (probeTupAppenderToSpilled.append(accessorProbe, i)) {
+                            break;
+                        } 
+                        probeWrite(pid, buff);
+                        buff.clear();
+                        needToClear = true;
+                    }
+                } else { //pid is Resident
+                    while (true) {
+                        if (probeTupAppenderToResident.append(accessorProbe, i)){
+                            break;
+                        }
+                        inMemJoiner.join(probeResBuff, writer);
+                        probeTupAppenderToResident.reset(probeResBuff, true);
+                    }
+
+                }
+                probePSizeInTups[pid]++;
+            }
+
+        }
+
+    }
+
+    public void closeProbe(IFrameWriter writer) throws HyracksDataException { //We do NOT join the spilled partitions here, that decision is made at the descriptor level (which join technique to use)
+        inMemJoiner.join(probeResBuff, writer);
+        inMemJoiner.closeJoin(writer);
+
+        for (int pid = pStatus.nextSetBit(0); pid >= 0; pid = pStatus.nextSetBit(pid + 1)) {
+            ByteBuffer buff = sPartBuffs[curPBuff[pid]];
+            accessorProbe.reset(buff);
+            if (accessorProbe.getTupleCount() > 0) {
+                probeWrite(pid, buff);
+            }
+            closeProbeWriter(pid);
+        }
+    }
+
+    private void probeWrite(int pid, ByteBuffer buff) throws HyracksDataException {
+        RunFileWriter pWriter = probeRFWriters[pid];
+        if (pWriter == null) {
+            FileReference file = ctx.createManagedWorkspaceFile(rel1Name);
+            pWriter = new RunFileWriter(file, ctx.getIOManager());
+            pWriter.open();
+            probeRFWriters[pid] = pWriter;
+        }
+        pWriter.nextFrame(buff);
+    }
+
+    private void closeProbeWriter(int pid) throws HyracksDataException {
+        RunFileWriter writer = probeRFWriters[pid];
+        if (writer != null) {
+            writer.close();
+        }
+    }
+
+    public RunFileReader getBuildRFReader(int pid) throws HyracksDataException {
+        return ((buildRFWriters[pid] == null) ? null : (buildRFWriters[pid]).createReader());
+    }
+
+    public long getBuildPartitionSize(int pid) {
+        return ((buildRFWriters[pid] == null) ? 0 : buildRFWriters[pid].getFileSize());
+    }
+
+    public int getBuildPartitionSizeInTup(int pid) {
+        return (buildPSizeInTups[pid]);
+    }
+
+    public RunFileReader getProbeRFReader(int pid) throws HyracksDataException {
+        return ((probeRFWriters[pid] == null) ? null : (probeRFWriters[pid]).createReader());
+    }
+
+    public long getProbePartitionSize(int pid) {
+        return ((probeRFWriters[pid] == null) ? 0 : probeRFWriters[pid].getFileSize());
+    }
+
+    public int getProbePartitionSizeInTup(int pid) {
+        return (probePSizeInTups[pid]);
+    }
+
+    public int getMaxBuildPartitionSize() {
+        int max = buildPSizeInTups[0];
+        for (int i = 1; i < buildPSizeInTups.length; i++) {
+            if (buildPSizeInTups[i] > max) {
+                max = buildPSizeInTups[i];
+            }
+        }
+        return max;
+    }
+
+    public int getMaxProbePartitionSize() {
+        int max = probePSizeInTups[0];
+        for (int i = 1; i < probePSizeInTups.length; i++) {
+            if (probePSizeInTups[i] > max) {
+                max = probePSizeInTups[i];
+            }
+        }
+        return max;
+    }
+
+    public BitSet getPartitinStatus() {
+        return pStatus;
+    }
+
+    public String debugGetStats() {
+        int numOfResidentPartitions = 0;
+        int numOfSpilledPartitions = 0;
+        double sumOfBuildSpilledSizes = 0;
+        double sumOfProbeSpilledSizes = 0;
+        int numOfInMemTups = 0;
+        for (int i = 0; i < numOfPartitions; i++) {
+            if (pStatus.get(i)) { //Spilled
+                numOfSpilledPartitions++;
+                sumOfBuildSpilledSizes += buildPSizeInTups[i];
+                sumOfProbeSpilledSizes += probePSizeInTups[i];
+            } else { //Resident
+                numOfResidentPartitions++;
+                numOfInMemTups += buildPSizeInTups[i];
+            }
+        }
+
+        double avgBuildSpSz = sumOfBuildSpilledSizes / numOfSpilledPartitions;
+        double avgProbeSpSz = sumOfProbeSpilledSizes / numOfSpilledPartitions;
+        String s = "Resident Partitions:\t" + numOfResidentPartitions + "\nSpilled Partitions:\t"
+                + numOfSpilledPartitions + "\nAvg Build Spilled Size:\t" + avgBuildSpSz + "\nAvg Probe Spilled Size:\t"
+                + avgProbeSpSz + "\nIn-Memory Tups:\t" + numOfInMemTups + "\nNum of Free Buffers:\t"
+                + freeFramesCounter;
+        return s;
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..3a7ee2c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -0,0 +1,635 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.join;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+
+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.IBinaryHashFunctionFamily;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+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.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFamily;
+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.common.comm.io.FrameTupleAccessor;
+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.common.data.partition.FieldHashPartitionComputerFamily;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.RepartitionComputerGeneratorFactory;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
+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.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.structures.ISerializableTable;
+import edu.uci.ics.hyracks.dataflow.std.structures.SerializableHashTable;
+
+/**
+ * @author pouria
+ *         This class guides the joining process, and switches between different
+ *         joining techniques, w.r.t the implemented optimizations and skew in size of the
+ *         partitions.
+ *         - Operator overview:
+ *         Assume we are trying to do (R Join S), with M buffers available, while we have an estimate on the size
+ *         of R (in terms of buffers). HHJ (Hybrid Hash Join) has two main phases: Build and Probe, where in our implementation Probe phase
+ *         can apply HHJ recursively, based on the value of M and size of R and S. HHJ phases proceed as follow:
+ *         BUILD:
+ *         Calculate number of partitions (Based on the size of R, fudge factor and M) [See Shapiro's paper for the detailed discussion].
+ *         Initialize the build phase (one frame per partition, all partitions considered resident at first)
+ *         Read tuples of R, frame by frame, and hash each tuple (based on a given hash function) to find
+ *         its target partition and try to append it to that partition:
+ *         If target partition's buffer is full, try to allocate a new buffer for it.
+ *         if no free buffer is available, find the largest resident partition and spill it. Using its freed
+ *         buffers after spilling, allocate a new buffer for the target partition.
+ *         Being done with R, close the build phase. (During closing we write the very last buffer of each
+ *         spilled partition to the disk, and we do partition tuning, where we try to bring back as many buffers, belonging to
+ *         spilled partitions as possible into memory, based on the free buffers - We will stop at the point where remaining free buffers is not enough
+ *         for reloading an entire partition back into memory)
+ *         Create the hash table for the resident partitions (basically we create an in-memory hash join here)
+ *         PROBE:
+ *         Initialize the probe phase on S (mainly allocate one buffer per spilled partition, and one buffer
+ *         for the whole resident partitions)
+ *         Read tuples of S, frame by frame and hash each tuple T to its target partition P
+ *         if P is a resident partition, pass T to the in-memory hash join and generate the output record,
+ *         if any matching(s) record found
+ *         if P is spilled, write T to the dedicated buffer for P (on the probe side)
+ *         Once scanning of S is done, we try to join partition pairs (Ri, Si) of the spilled partitions:
+ *         if any of Ri or Si is smaller than M, then we simply use an in-memory hash join to join them
+ *         otherwise we apply HHJ recursively:
+ *         if after applying HHJ recursively, we do not gain enough size reduction (max size of the
+ *         resulting partitions were more than 80% of the initial Ri,Si size) then we switch to
+ *         nested loop join for joining.
+ *         (At each step of partition-pair joining, we consider role reversal, which means if size of Si were
+ *         greater than Ri, then we make sure that we switch the roles of build/probe between them)
+ */
+
+public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final int BUILD_AND_PARTITION_ACTIVITY_ID = 0;
+    private static final int PARTITION_AND_JOIN_ACTIVITY_ID = 1;
+
+    private static final long serialVersionUID = 1L;
+    private static final double NLJ_SWITCH_THRESHOLD = 0.8;
+
+    private static final String PROBE_REL = "RelR";
+    private static final String BUILD_REL = "RelS";
+
+    private final int memsize;
+    private final int inputsize0;
+    private final double fudgeFactor;
+    private final int[] probeKeys;
+    private final int[] buildKeys;
+    private final IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories;
+    private final IBinaryComparatorFactory[] comparatorFactories; //For in-mem HJ
+    private final ITuplePairComparatorFactory tuplePairComparatorFactory0; //For NLJ in probe
+    private final ITuplePairComparatorFactory tuplePairComparatorFactory1; //For NLJ in probe
+
+    private final boolean isLeftOuter;
+    private final INullWriterFactory[] nullWriterFactories1;
+
+    public OptimizedHybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+            double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor,
+            ITuplePairComparatorFactory tupPaircomparatorFactory0,
+            ITuplePairComparatorFactory tupPaircomparatorFactory1, boolean isLeftOuter,
+            INullWriterFactory[] nullWriterFactories1) throws HyracksDataException {
+
+        super(spec, 2, 1);
+        this.memsize = memsize;
+        this.inputsize0 = inputsize0;
+        this.fudgeFactor = factor;
+        this.probeKeys = keys0;
+        this.buildKeys = keys1;
+        this.hashFunctionGeneratorFactories = hashFunctionGeneratorFactories;
+        this.comparatorFactories = comparatorFactories;
+        this.tuplePairComparatorFactory0 = tupPaircomparatorFactory0;
+        this.tuplePairComparatorFactory1 = tupPaircomparatorFactory1;
+        recordDescriptors[0] = recordDescriptor;
+        this.isLeftOuter = isLeftOuter;
+        this.nullWriterFactories1 = nullWriterFactories1;
+
+    }
+
+    public OptimizedHybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+            double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor,
+            ITuplePairComparatorFactory tupPaircomparatorFactory0, ITuplePairComparatorFactory tupPaircomparatorFactory1)
+            throws HyracksDataException {
+
+        super(spec, 2, 1);
+        this.memsize = memsize;
+        this.inputsize0 = inputsize0;
+        this.fudgeFactor = factor;
+        this.probeKeys = keys0;
+        this.buildKeys = keys1;
+        this.hashFunctionGeneratorFactories = hashFunctionGeneratorFactories;
+        this.comparatorFactories = comparatorFactories;
+        this.tuplePairComparatorFactory0 = tupPaircomparatorFactory0;
+        this.tuplePairComparatorFactory1 = tupPaircomparatorFactory1;
+        recordDescriptors[0] = recordDescriptor;
+        this.isLeftOuter = false;
+        this.nullWriterFactories1 = null;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        ActivityId buildAid = new ActivityId(odId, BUILD_AND_PARTITION_ACTIVITY_ID);
+        ActivityId probeAid = new ActivityId(odId, PARTITION_AND_JOIN_ACTIVITY_ID);
+        PartitionAndBuildActivityNode phase1 = new PartitionAndBuildActivityNode(buildAid, probeAid);
+        ProbeAndJoinActivityNode phase2 = new ProbeAndJoinActivityNode(probeAid, buildAid);
+
+        builder.addActivity(this, phase1);
+        builder.addSourceEdge(0, phase1, 0);
+
+        builder.addActivity(this, phase2);
+        builder.addSourceEdge(1, phase2, 0);
+
+        builder.addBlockingEdge(phase1, phase2);
+
+        builder.addTargetEdge(0, phase2, 0);
+
+    }
+
+    //memorySize is the memory for join (we have already excluded the 2 buffers for in/out)
+    private int getNumberOfPartitions(int memorySize, int buildSize, double factor, int nPartitions)
+            throws HyracksDataException {
+        int numberOfPartitions = 0;
+        if (memorySize <= 1) {
+            throw new HyracksDataException("not enough memory is available for Hybrid Hash Join");
+        }
+        if (memorySize > buildSize) {
+            return 1; //We will switch to in-Mem HJ eventually
+        }
+        numberOfPartitions = (int) (Math.ceil((double) (buildSize * factor / nPartitions - memorySize)
+                / (double) (memorySize - 1)));
+        if (numberOfPartitions <= 0) {
+            numberOfPartitions = 1; //becomes in-memory hash join
+        }
+        if (numberOfPartitions > memorySize) {
+            numberOfPartitions = (int) Math.ceil(Math.sqrt(buildSize * factor / nPartitions));
+            return (numberOfPartitions < memorySize ? numberOfPartitions : memorySize);
+        }
+        return numberOfPartitions;
+    }
+
+    public static class BuildAndPartitionTaskState extends AbstractStateObject {
+
+        private int memForJoin;
+        private int numOfPartitions;
+        private OptimizedHybridHashJoin hybridHJ;
+
+        public BuildAndPartitionTaskState() {
+        }
+
+        private BuildAndPartitionTaskState(JobId jobId, TaskId taskId) {
+            super(jobId, taskId);
+        }
+
+        @Override
+        public void toBytes(DataOutput out) throws IOException {
+
+        }
+
+        @Override
+        public void fromBytes(DataInput in) throws IOException {
+
+        }
+
+    }
+
+    /*
+     * Build phase of Hybrid Hash Join:
+     * Creating an instance of Hybrid Hash Join, using Shapiro's formula
+     * to get the optimal number of partitions, build relation is read and
+     * partitioned, and hybrid hash join instance gets ready for the probing.
+     * (See OptimizedHybridHashJoin for the details on different steps)
+     */
+    private class PartitionAndBuildActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId probeAid;
+
+        public PartitionAndBuildActivityNode(ActivityId id, ActivityId probeAid) {
+            super(id);
+            this.probeAid = probeAid;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+
+            final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(probeAid, 0);
+
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            for (int i = 0; i < comparatorFactories.length; i++) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+
+            final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
+            if (isLeftOuter) {
+                for (int i = 0; i < nullWriterFactories1.length; i++) {
+                    nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
+                }
+            }
+
+            IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
+                private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(ctx.getJobletContext()
+                        .getJobId(), new TaskId(getActivityId(), partition));
+
+                ITuplePartitionComputer probeHpc = new FieldHashPartitionComputerFamily(probeKeys,
+                        hashFunctionGeneratorFactories).createPartitioner(0);
+                ITuplePartitionComputer buildHpc = new FieldHashPartitionComputerFamily(buildKeys,
+                        hashFunctionGeneratorFactories).createPartitioner(0);
+
+                @Override
+                public void open() throws HyracksDataException {
+                    if (memsize <= 2) { //Dedicated buffers: One buffer to read and one buffer for output
+                        throw new HyracksDataException("not enough memory for Hybrid Hash Join");
+                    }
+                    state.memForJoin = memsize - 2;
+                    state.numOfPartitions = getNumberOfPartitions(state.memForJoin, inputsize0, fudgeFactor,
+                            nPartitions);
+                    state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
+                            PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
+                            buildHpc);
+                    state.hybridHJ.initBuild();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    state.hybridHJ.build(buffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    state.hybridHJ.closeBuild();
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                }
+
+            };
+            return op;
+        }
+    }
+
+    /*
+     * Probe phase of Hybrid Hash Join:
+     * Reading the probe side and partitioning it, resident tuples get
+     * joined with the build side residents (through formerly created HybridHashJoin in the build phase)
+     * and spilled partitions get written to run files. During the close() call, pairs of spilled partition
+     * (build side spilled partition and its corresponding probe side spilled partition) join, by applying
+     * Hybrid Hash Join recursively on them.
+     */
+    private class ProbeAndJoinActivityNode extends AbstractActivityNode {
+
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId buildAid;
+
+        public ProbeAndJoinActivityNode(ActivityId id, ActivityId buildAid) {
+            super(id);
+            this.buildAid = buildAid;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+
+            final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
+            final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+            final ITuplePairComparator nljComparator0 = tuplePairComparatorFactory0.createTuplePairComparator(ctx);
+            final ITuplePairComparator nljComparator1 = tuplePairComparatorFactory1.createTuplePairComparator(ctx);
+
+            for (int i = 0; i < comparatorFactories.length; i++) {
+                comparators[i] = comparatorFactories[i].createBinaryComparator();
+            }
+
+            final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
+            if (isLeftOuter) {
+                for (int i = 0; i < nullWriterFactories1.length; i++) {
+                    nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
+                }
+            }
+
+            IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+                private BuildAndPartitionTaskState state;
+                private ByteBuffer rPartbuff = ctx.allocateFrame();
+
+                private ITuplePartitionComputerFamily hpcf0 = new FieldHashPartitionComputerFamily(probeKeys,
+                        hashFunctionGeneratorFactories);
+                private ITuplePartitionComputerFamily hpcf1 = new FieldHashPartitionComputerFamily(buildKeys,
+                        hashFunctionGeneratorFactories);
+
+                private ITuplePartitionComputer hpcRep0;
+                private ITuplePartitionComputer hpcRep1;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    state = (BuildAndPartitionTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
+                            BUILD_AND_PARTITION_ACTIVITY_ID), partition));
+
+                    writer.open();
+                    state.hybridHJ.initProbe();
+
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    state.hybridHJ.probe(buffer, writer);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    writer.fail();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+
+                    state.hybridHJ.closeProbe(writer);
+
+                    BitSet partitionStatus = state.hybridHJ.getPartitinStatus();
+                    hpcRep0 = new RepartitionComputerGeneratorFactory(state.numOfPartitions, hpcf0)
+                            .createPartitioner(0);
+                    hpcRep1 = new RepartitionComputerGeneratorFactory(state.numOfPartitions, hpcf1)
+                            .createPartitioner(0);
+
+                    rPartbuff.clear();
+                    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)
+                            continue;
+                        }
+                        int bSize = state.hybridHJ.getBuildPartitionSizeInTup(pid);
+                        int pSize = state.hybridHJ.getProbePartitionSizeInTup(pid);
+                        int beforeMax = (bSize > pSize) ? bSize : pSize;
+                        joinPartitionPair(state.hybridHJ, bReader, pReader, pid, beforeMax, 1);
+
+                    }
+                    writer.close();
+                }
+
+                private void joinPartitionPair(OptimizedHybridHashJoin ohhj, RunFileReader buildSideReader,
+                        RunFileReader probeSideReader, int pid, int beforeMax, int level) throws HyracksDataException {
+                    ITuplePartitionComputer probeHpc = new FieldHashPartitionComputerFamily(probeKeys,
+                            hashFunctionGeneratorFactories).createPartitioner(level);
+                    ITuplePartitionComputer buildHpc = new FieldHashPartitionComputerFamily(buildKeys,
+                            hashFunctionGeneratorFactories).createPartitioner(level);
+
+                    long buildPartSize = ohhj.getBuildPartitionSize(pid) / ctx.getFrameSize();
+                    long probePartSize = ohhj.getProbePartitionSize(pid) / ctx.getFrameSize();
+
+                    //Apply in-Mem HJ if possible
+                    if ((buildPartSize < state.memForJoin) || (probePartSize < state.memForJoin)) {
+                        int tabSize = -1;
+                        if (buildPartSize < probePartSize) {
+                            tabSize = ohhj.getBuildPartitionSizeInTup(pid);
+                            if (tabSize == 0) {
+                                throw new HyracksDataException(
+                                        "Trying to join an empty partition. Invalid table size for inMemoryHashJoin.");
+                            }
+                            //Build Side is smaller
+                            applyInMemHashJoin(probeKeys, buildKeys, tabSize, probeRd, buildRd, hpcRep1, hpcRep0,
+                                    buildSideReader, probeSideReader);
+
+                        } else { //Role Reversal
+                            tabSize = ohhj.getProbePartitionSizeInTup(pid);
+                            if (tabSize == 0) {
+                                throw new HyracksDataException(
+                                        "Trying to join an empty partition. Invalid table size for inMemoryHashJoin.");
+                            }
+                            //Probe Side is smaller
+                            applyInMemHashJoin(buildKeys, probeKeys, tabSize, buildRd, probeRd, hpcRep0, hpcRep1,
+                                    probeSideReader, buildSideReader);
+                        }
+                    }
+                    //Apply (Recursive) HHJ
+                    else {
+                        OptimizedHybridHashJoin rHHj;
+                        if (buildPartSize < probePartSize) { //Build Side is smaller
+
+                            int n = getNumberOfPartitions(state.memForJoin, (int) buildPartSize, fudgeFactor,
+                                    nPartitions);
+                            rHHj = new OptimizedHybridHashJoin(ctx, state.memForJoin, n, PROBE_REL, BUILD_REL,
+                                    probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc, buildHpc);
+
+                            buildSideReader.open();
+                            rHHj.initBuild();
+                            rPartbuff.clear();
+                            while (buildSideReader.nextFrame(rPartbuff)) {
+                                rHHj.build(rPartbuff);
+                            }
+
+                            rHHj.closeBuild();
+
+                            probeSideReader.open();
+                            rHHj.initProbe();
+                            rPartbuff.clear();
+                            while (probeSideReader.nextFrame(rPartbuff)) {
+                                rHHj.probe(rPartbuff, writer);
+                            }
+                            rHHj.closeProbe(writer);
+
+                            int maxAfterBuildSize = rHHj.getMaxBuildPartitionSize();
+                            int maxAfterProbeSize = rHHj.getMaxProbePartitionSize();
+                            int afterMax = (maxAfterBuildSize > maxAfterProbeSize) ? maxAfterBuildSize
+                                    : maxAfterProbeSize;
+
+                            BitSet rPStatus = rHHj.getPartitinStatus();
+                            if (afterMax < NLJ_SWITCH_THRESHOLD * beforeMax) {
+                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+                                    RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
+                                    RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
+
+                                    if (rbrfw == null || rprfw == null) {
+                                        continue;
+                                    }
+
+                                    joinPartitionPair(rHHj, rbrfw, rprfw, rPid, afterMax, (level + 1));
+                                }
+
+                            } else { //Switch to NLJ (Further recursion seems not to be useful)
+                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+                                    RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
+                                    RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
+
+                                    if (rbrfw == null || rprfw == null) {
+                                        continue;
+                                    }
+
+                                    int buildSideInTups = rHHj.getBuildPartitionSizeInTup(rPid);
+                                    int probeSideInTups = rHHj.getProbePartitionSizeInTup(rPid);
+                                    if (buildSideInTups < probeSideInTups) {
+                                        applyNestedLoopJoin(probeRd, buildRd, state.memForJoin, rbrfw, rprfw,
+                                                nljComparator0);
+                                    } else {
+                                        applyNestedLoopJoin(buildRd, probeRd, state.memForJoin, rprfw, rbrfw,
+                                                nljComparator1);
+                                    }
+                                }
+                            }
+                        } else { //Role Reversal (Probe Side is smaller)
+                            int n = getNumberOfPartitions(state.memForJoin, (int) probePartSize, fudgeFactor,
+                                    nPartitions);
+                            rHHj = new OptimizedHybridHashJoin(ctx, state.memForJoin, n, BUILD_REL, PROBE_REL,
+                                    buildKeys, probeKeys, comparators, buildRd, probeRd, buildHpc, probeHpc);
+
+                            probeSideReader.open();
+                            rHHj.initBuild();
+                            rPartbuff.clear();
+                            while (probeSideReader.nextFrame(rPartbuff)) {
+                                rHHj.build(rPartbuff);
+                            }
+                            rHHj.closeBuild();
+                            rHHj.initProbe();
+                            buildSideReader.open();
+                            rPartbuff.clear();
+                            while (buildSideReader.nextFrame(rPartbuff)) {
+                                rHHj.probe(rPartbuff, writer);
+                            }
+                            rHHj.closeProbe(writer);
+                            int maxAfterBuildSize = rHHj.getMaxBuildPartitionSize();
+                            int maxAfterProbeSize = rHHj.getMaxProbePartitionSize();
+                            int afterMax = (maxAfterBuildSize > maxAfterProbeSize) ? maxAfterBuildSize
+                                    : maxAfterProbeSize;
+                            BitSet rPStatus = rHHj.getPartitinStatus();
+
+                            if (afterMax < NLJ_SWITCH_THRESHOLD * beforeMax) {
+                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+                                    RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
+                                    RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
+
+                                    if (rbrfw == null || rprfw == null) {
+                                        continue;
+                                    }
+
+                                    joinPartitionPair(rHHj, rprfw, rbrfw, rPid, afterMax, (level + 1));
+                                }
+                            } else { //Switch to NLJ (Further recursion seems not to be effective)
+                                for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+                                    RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
+                                    RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
+
+                                    if (rbrfw == null || rprfw == null) {
+                                        continue;
+                                    }
+
+                                    long buildSideSize = rbrfw.getFileSize();
+                                    long probeSideSize = rprfw.getFileSize();
+                                    if (buildSideSize > probeSideSize) {
+                                        applyNestedLoopJoin(buildRd, probeRd, state.memForJoin, rbrfw, rprfw,
+                                                nljComparator1);
+                                    } else {
+                                        applyNestedLoopJoin(probeRd, buildRd, state.memForJoin, rprfw, rbrfw,
+                                                nljComparator0);
+                                    }
+                                }
+                            }
+                        }
+                        buildSideReader.close();
+                        probeSideReader.close();
+                    }
+                }
+
+                private void applyInMemHashJoin(int[] bKeys, int[] pKeys, int tabSize, RecordDescriptor buildRDesc,
+                        RecordDescriptor probeRDesc, ITuplePartitionComputer hpcRepLarger,
+                        ITuplePartitionComputer hpcRepSmaller, RunFileReader bReader, RunFileReader pReader)
+                        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);
+
+                    bReader.open();
+                    rPartbuff.clear();
+                    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);
+                        joiner.build(copyBuffer);
+                        rPartbuff.clear();
+                    }
+                    bReader.close();
+                    rPartbuff.clear();
+                    // probe
+                    pReader.open();
+                    while (pReader.nextFrame(rPartbuff)) {
+                        joiner.join(rPartbuff, writer);
+                        rPartbuff.clear();
+                    }
+                    pReader.close();
+                    joiner.closeJoin(writer);
+                }
+
+                private void applyNestedLoopJoin(RecordDescriptor outerRd, RecordDescriptor innerRd, int memorySize,
+                        RunFileReader outerReader, RunFileReader innerReader, ITuplePairComparator nljComparator)
+                        throws HyracksDataException {
+
+                    NestedLoopJoin nlj = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), outerRd),
+                            new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize);
+
+                    ByteBuffer cacheBuff = ctx.allocateFrame();
+                    innerReader.open();
+                    while (innerReader.nextFrame(cacheBuff)) {
+                        FrameUtils.makeReadable(cacheBuff);
+                        nlj.cache(cacheBuff);
+                        cacheBuff.clear();
+                    }
+                    nlj.closeCache();
+
+                    ByteBuffer joinBuff = ctx.allocateFrame();
+                    outerReader.open();
+
+                    while (outerReader.nextFrame(joinBuff)) {
+                        FrameUtils.makeReadable(joinBuff);
+                        nlj.join(joinBuff, writer);
+                        joinBuff.clear();
+                    }
+
+                    nlj.closeJoin(writer);
+                    outerReader.close();
+                    innerReader.close();
+                }
+            };
+            return op;
+        }
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
new file mode 100644
index 0000000..30aae7f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.map;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public class DeserializedMapperOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private class MapperOperator implements IOpenableDataWriterOperator {
+        private IDeserializedMapper mapper;
+        private IOpenableDataWriter<Object[]> writer;
+
+        @Override
+        public void close() throws HyracksDataException {
+            writer.close();
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+            writer.fail();
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            mapper = mapperFactory.createMapper();
+            writer.open();
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            if (index != 0) {
+                throw new IllegalArgumentException();
+            }
+            this.writer = writer;
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            mapper.map(data, writer);
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private final IDeserializedMapperFactory mapperFactory;
+
+    public DeserializedMapperOperatorDescriptor(IOperatorDescriptorRegistry spec,
+            IDeserializedMapperFactory mapperFactory, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.mapperFactory = mapperFactory;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new DeserializedOperatorNodePushable(ctx, new MapperOperator(),
+                recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/IDeserializedMapper.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/IDeserializedMapper.java
new file mode 100644
index 0000000..fe9ab1c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/IDeserializedMapper.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.map;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IDeserializedMapper {
+    public void map(Object[] data, IDataWriter<Object[]> writer) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/IDeserializedMapperFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/IDeserializedMapperFactory.java
new file mode 100644
index 0000000..6fadfdc
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/IDeserializedMapperFactory.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.map;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IDeserializedMapperFactory extends Serializable {
+    public IDeserializedMapper createMapper() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/ReflectionBasedDeserializedMapperFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/ReflectionBasedDeserializedMapperFactory.java
new file mode 100644
index 0000000..d207a5c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/ReflectionBasedDeserializedMapperFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.map;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ReflectionBasedDeserializedMapperFactory implements IDeserializedMapperFactory {
+    private static final long serialVersionUID = 1L;
+
+    private final Class<? extends IDeserializedMapper> mapperClass;
+
+    public ReflectionBasedDeserializedMapperFactory(Class<? extends IDeserializedMapper> mapperClass) {
+        this.mapperClass = mapperClass;
+    }
+
+    @Override
+    public IDeserializedMapper createMapper() throws HyracksDataException {
+        try {
+            return mapperClass.newInstance();
+        } catch (InstantiationException e) {
+            throw new HyracksDataException(e);
+        } catch (IllegalAccessException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/SamplerDeserializedMapperFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/SamplerDeserializedMapperFactory.java
new file mode 100644
index 0000000..bf05106
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/SamplerDeserializedMapperFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.map;
+
+import java.util.Random;
+
+import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class SamplerDeserializedMapperFactory implements IDeserializedMapperFactory {
+    private static final long serialVersionUID = 1L;
+    private final float sampleRatio;
+
+    public SamplerDeserializedMapperFactory(float sampleRatio) {
+        this.sampleRatio = sampleRatio;
+    }
+
+    @Override
+    public IDeserializedMapper createMapper() throws HyracksDataException {
+        return new IDeserializedMapper() {
+            private Random random = new Random();
+
+            @Override
+            public void map(Object[] data, IDataWriter<Object[]> writer) throws HyracksDataException {
+                if (random.nextFloat() <= sampleRatio) {
+                    writer.writeData(data);
+                }
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
new file mode 100644
index 0000000..e22db5c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.misc;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+
+public class ConstantTupleSourceOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    private int[] fieldSlots;
+    private byte[] tupleData;
+    private int tupleSize;
+
+    public ConstantTupleSourceOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc, int[] fieldSlots,
+            byte[] tupleData, int tupleSize) {
+        super(spec, 0, 1);
+        this.tupleData = tupleData;
+        this.fieldSlots = fieldSlots;
+        this.tupleSize = tupleSize;
+        recordDescriptors[0] = recDesc;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+        return new ConstantTupleSourceOperatorNodePushable(ctx, fieldSlots, tupleData, tupleSize);
+    }
+}
\ No newline at end of file
diff --git a/hyracks/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
new file mode 100644
index 0000000..ab9b44c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.misc;
+
+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.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 {
+    private IHyracksTaskContext ctx;
+
+    private int[] fieldSlots;
+    private byte[] tupleData;
+    private int tupleSize;
+
+    public ConstantTupleSourceOperatorNodePushable(IHyracksTaskContext ctx, int[] fieldSlots, byte[] tupleData,
+            int tupleSize) {
+        super();
+        this.fieldSlots = fieldSlots;
+        this.tupleData = tupleData;
+        this.tupleSize = tupleSize;
+        this.ctx = ctx;
+    }
+
+    @Override
+    public void initialize() throws HyracksDataException {
+        ByteBuffer writeBuffer = ctx.allocateFrame();
+        FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+        appender.reset(writeBuffer, true);
+        if (fieldSlots != null && tupleData != null && tupleSize > 0)
+            appender.append(fieldSlots, tupleData, 0, tupleSize);
+        writer.open();
+        try {
+            FrameUtils.flushFrame(writeBuffer, writer);
+        } catch (Exception e) {
+            writer.fail();
+            throw new HyracksDataException(e);
+        } finally {
+            writer.close();
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/IdentityOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/IdentityOperatorDescriptor.java
new file mode 100644
index 0000000..b58b1c2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/IdentityOperatorDescriptor.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.misc;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class IdentityOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public IdentityOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor rDesc) {
+        super(spec, 1, 1);
+        recordDescriptors[0] = rDesc;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+            throws HyracksDataException {
+        return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+            @Override
+            public void open() throws HyracksDataException {
+                writer.open();
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                writer.nextFrame(buffer);
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                writer.fail();
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                writer.close();
+            }
+        };
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..2d2f074
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/LimitOperatorDescriptor.java
@@ -0,0 +1,96 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.misc;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.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.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.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class LimitOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private final int outputLimit;
+
+    public LimitOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor rDesc, int outputLimit) {
+        super(spec, 1, 1);
+        recordDescriptors[0] = rDesc;
+        this.outputLimit = outputLimit;
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+            throws HyracksDataException {
+
+        return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+            private FrameTupleAccessor fta;
+            private int currentSize;
+            private boolean finished;
+
+            @Override
+            public void open() throws HyracksDataException {
+                fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptors[0]);
+                currentSize = 0;
+                finished = false;
+                writer.open();
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                if (!finished) {
+                    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);
+                        int copyCount = outputLimit - currentSize;
+                        for (int i = 0; i < copyCount; i++) {
+                            partialAppender.append(fta, i);
+                            currentSize++;
+                        }
+                        FrameUtils.makeReadable(b);
+                        FrameUtils.flushFrame(b, writer);
+                        finished = true;
+                    } else {
+                        FrameUtils.flushFrame(buffer, writer);
+                        currentSize += count;
+                    }
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                writer.fail();
+
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                writer.close();
+            }
+        };
+    }
+
+}
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
new file mode 100644
index 0000000..ba8c00c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
@@ -0,0 +1,168 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.misc;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+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.IRecordDescriptorProvider;
+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.IOperatorDescriptorRegistry;
+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.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 class MaterializingOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final static int MATERIALIZER_ACTIVITY_ID = 0;
+    private final static int READER_ACTIVITY_ID = 1;
+
+    public MaterializingOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        MaterializerActivityNode ma = new MaterializerActivityNode(new ActivityId(odId, MATERIALIZER_ACTIVITY_ID));
+        ReaderActivityNode ra = new ReaderActivityNode(new ActivityId(odId, READER_ACTIVITY_ID));
+
+        builder.addActivity(this, ma);
+        builder.addSourceEdge(0, ma, 0);
+
+        builder.addActivity(this, ra);
+        builder.addTargetEdge(0, ra, 0);
+
+        builder.addBlockingEdge(ma, ra);
+    }
+
+    public static class MaterializerTaskState extends AbstractStateObject {
+        private RunFileWriter out;
+
+        public MaterializerTaskState() {
+        }
+
+        private MaterializerTaskState(JobId jobId, TaskId taskId) {
+            super(jobId, taskId);
+        }
+
+        @Override
+        public void toBytes(DataOutput out) throws IOException {
+
+        }
+
+        @Override
+        public void fromBytes(DataInput in) throws IOException {
+
+        }
+    }
+
+    private final class MaterializerActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public MaterializerActivityNode(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            return new AbstractUnaryInputSinkOperatorNodePushable() {
+                private MaterializerTaskState state;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    state = new MaterializerTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(),
+                            partition));
+                    FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+                            MaterializingOperatorDescriptor.class.getSimpleName());
+                    state.out = new RunFileWriter(file, ctx.getIOManager());
+                    state.out.open();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    state.out.nextFrame(buffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    state.out.close();
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                }
+            };
+        }
+    }
+
+    private final class ReaderActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public ReaderActivityNode(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            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));
+                    RunFileReader in = state.out.createReader();
+                    writer.open();
+                    try {
+                        in.open();
+                        while (in.nextFrame(frame)) {
+                            frame.flip();
+                            writer.nextFrame(frame);
+                            frame.clear();
+                        }
+                        in.close();
+                    } catch (Exception e) {
+                        writer.fail();
+                        throw new HyracksDataException(e);
+                    } finally {
+                        writer.close();
+                    }
+                }
+
+                @Override
+                public void deinitialize() throws HyracksDataException {
+                }
+            };
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
new file mode 100644
index 0000000..7c32c60
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.misc;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+
+public class NullSinkOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public NullSinkOperatorDescriptor(IOperatorDescriptorRegistry spec) {
+        super(spec, 1, 0);
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new AbstractUnaryInputSinkOperatorNodePushable() {
+            @Override
+            public void open() throws HyracksDataException {
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
new file mode 100644
index 0000000..47a8616
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.misc;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.util.StringSerializationUtils;
+
+public class PrinterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public PrinterOperatorDescriptor(IOperatorDescriptorRegistry spec) {
+        super(spec, 1, 0);
+    }
+
+    private class PrinterOperator implements IOpenableDataWriterOperator {
+        @Override
+        public void open() throws HyracksDataException {
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+        }
+
+        @Override
+        public void writeData(Object[] data) throws HyracksDataException {
+            for (int i = 0; i < data.length; ++i) {
+                System.err.print(StringSerializationUtils.toString(data[i]));
+                System.err.print(", ");
+            }
+            System.err.println();
+        }
+
+        @Override
+        public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+            throw new IllegalArgumentException();
+        }
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+        return new DeserializedOperatorNodePushable(ctx, new PrinterOperator(),
+                recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..231a8d7
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
@@ -0,0 +1,67 @@
+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.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.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.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
+
+public class SplitOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public SplitOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor rDesc, int outputArity) {
+        super(spec, 1, outputArity);
+        for (int i = 0; i < outputArity; i++) {
+            recordDescriptors[i] = rDesc;
+        }
+    }
+
+    @Override
+    public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+            final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+            throws HyracksDataException {
+        return new AbstractUnaryInputOperatorNodePushable() {
+            private final IFrameWriter[] writers = new IFrameWriter[outputArity];
+
+            @Override
+            public void close() throws HyracksDataException {
+                for (IFrameWriter writer : writers) {
+                    writer.close();
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                for (IFrameWriter writer : writers) {
+                    writer.fail();
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer bufferAccessor) throws HyracksDataException {
+                for (IFrameWriter writer : writers) {
+                    FrameUtils.flushFrame(bufferAccessor, writer);
+                }
+            }
+
+            @Override
+            public void open() throws HyracksDataException {
+                for (IFrameWriter writer : writers) {
+                    writer.open();
+                }
+            }
+
+            @Override
+            public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                writers[index] = writer;
+            }
+        };
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
new file mode 100644
index 0000000..aad0e66
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
@@ -0,0 +1,189 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.misc;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+
+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.IOpenableDataWriter;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+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.IOpenableDataWriterOperator;
+import edu.uci.ics.hyracks.dataflow.std.util.DeserializedOperatorNodePushable;
+
+public class SplitVectorOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final int COLLECT_ACTIVITY_ID = 0;
+    private static final int SPLIT_ACTIVITY_ID = 1;
+
+    public static class CollectTaskState extends AbstractStateObject {
+        private ArrayList<Object[]> buffer;
+
+        public CollectTaskState() {
+        }
+
+        private CollectTaskState(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 CollectActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public CollectActivity(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public ActivityId getActivityId() {
+            return id;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
+                private CollectTaskState state;
+
+                @Override
+                public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+                    throw new IllegalArgumentException();
+                }
+
+                @Override
+                public void open() throws HyracksDataException {
+                    state = new CollectTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(),
+                            partition));
+                    state.buffer = new ArrayList<Object[]>();
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void writeData(Object[] data) throws HyracksDataException {
+                    state.buffer.add(data);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+
+                }
+            };
+            return new DeserializedOperatorNodePushable(ctx, op, recordDescProvider.getInputRecordDescriptor(
+                    getActivityId(), 0));
+        }
+    }
+
+    private class SplitActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public SplitActivity(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
+                private IOpenableDataWriter<Object[]> writer;
+
+                private CollectTaskState state;
+
+                @Override
+                public void setDataWriter(int index, IOpenableDataWriter<Object[]> writer) {
+                    if (index != 0) {
+                        throw new IllegalArgumentException();
+                    }
+                    this.writer = writer;
+                }
+
+                @Override
+                public void open() throws HyracksDataException {
+                    state = (CollectTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
+                            COLLECT_ACTIVITY_ID), partition));
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                }
+
+                @Override
+                public void writeData(Object[] data) throws HyracksDataException {
+                    int n = state.buffer.size();
+                    int step = (int) Math.floor(n / (float) splits);
+                    writer.open();
+                    for (int i = 0; i < splits; ++i) {
+                        writer.writeData(state.buffer.get(step * (i + 1) - 1));
+                    }
+                    writer.close();
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    writer.fail();
+                }
+            };
+            return new DeserializedOperatorNodePushable(ctx, op, recordDescProvider.getOutputRecordDescriptor(
+                    getActivityId(), 0));
+        }
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    private final int splits;
+
+    public SplitVectorOperatorDescriptor(IOperatorDescriptorRegistry spec, int splits, RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.splits = splits;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        CollectActivity ca = new CollectActivity(new ActivityId(odId, COLLECT_ACTIVITY_ID));
+        SplitActivity sa = new SplitActivity(new ActivityId(odId, SPLIT_ACTIVITY_ID));
+
+        builder.addActivity(this, ca);
+        builder.addSourceEdge(0, ca, 0);
+
+        builder.addActivity(this, sa);
+        builder.addTargetEdge(0, sa, 0);
+
+        builder.addBlockingEdge(ca, sa);
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..0c569f2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
@@ -0,0 +1,717 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.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() {
+        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
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
new file mode 100644
index 0000000..2b9f7c9
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTNodeUtil.java
@@ -0,0 +1,233 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.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
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
new file mode 100644
index 0000000..be71b44
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -0,0 +1,183 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.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.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 class ExternalSortOperatorDescriptor extends AbstractOperatorDescriptor {
+    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 framesLimit;
+
+    public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
+        this(spec, framesLimit, sortFields, null, comparatorFactories, recordDescriptor);
+    }
+
+    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;
+    }
+
+    @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 FrameSorter 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 {
+
+        }
+    }
+
+    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], 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;
+        }
+    }
+
+    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;
+                    FrameSorter frameSorter = state.frameSorter;
+                    IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+                    for (int i = 0; i < comparatorFactories.length; ++i) {
+                        comparators[i] = comparatorFactories[i].createBinaryComparator();
+                    }
+                    int necessaryFrames = Math.min(runs.size() + 2, framesLimit);
+                    ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, frameSorter, runs, sortFields,
+                            comparators, recordDescriptors[0], necessaryFrames, writer);
+                    merger.process();
+                }
+            };
+            return op;
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..bcfdb89
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -0,0 +1,97 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.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;
+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;
+
+public class ExternalSortRunGenerator implements IFrameWriter {
+    private final IHyracksTaskContext ctx;
+    private final FrameSorter frameSorter;
+    private final List<IFrameReader> runs;
+    private final int maxSortFrames;
+
+    public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDesc, int framesLimit) {
+        this.ctx = ctx;
+        frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc);
+        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) {
+            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();
+        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());
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+    }
+
+    public FrameSorter getFrameSorter() {
+        return frameSorter;
+    }
+
+    public List<IFrameReader> getRuns() {
+        return runs;
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..b51132e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -0,0 +1,296 @@
+package edu.uci.ics.hyracks.dataflow.std.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;
+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.
+ */
+
+public class ExternalSortRunMerger {
+
+    private final IHyracksTaskContext ctx;
+    private final List<IFrameReader> runs;
+    private final int[] sortFields;
+    private final IBinaryComparator[] comparators;
+    private final RecordDescriptor recordDesc;
+    private final int framesLimit;
+    private final IFrameWriter writer;
+    private List<ByteBuffer> inFrames;
+    private ByteBuffer outFrame;
+    private FrameTupleAppender outFrameAppender;
+
+    private FrameSorter 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, FrameSorter frameSorter, List<IFrameReader> runs,
+            int[] sortFields, IBinaryComparator[] comparators, 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.recordDesc = recordDesc;
+        this.framesLimit = framesLimit;
+        this.writer = writer;
+        this.outputLimit = -1;
+    }
+
+    // Constructor for external sort with replacement selection
+    public ExternalSortRunMerger(IHyracksTaskContext ctx, int outputLimit, List<IFrameReader> runs, int[] sortFields,
+            IBinaryComparator[] comparators, RecordDescriptor recordDesc, int framesLimit, IFrameWriter writer) {
+        this.ctx = ctx;
+        this.runs = new LinkedList<IFrameReader>(runs);
+        this.sortFields = sortFields;
+        this.comparators = comparators;
+        this.recordDesc = recordDesc;
+        this.framesLimit = framesLimit;
+        this.writer = writer;
+        this.outputLimit = outputLimit;
+        this.currentSize = 0;
+        this.frameSorter = null;
+    }
+
+    public void process() throws HyracksDataException {
+        writer.open();
+        try {
+            if (runs.size() <= 0) {
+                if (frameSorter != null && frameSorter.getFrameCount() > 0) {
+                    frameSorter.flushFrames(writer);
+                }
+                /** 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(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);
+                        }
+                        merge(mergeResultWriter, runCursors);
+                        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);
+                    }
+                    merge(writer, runCursors);
+                }
+            }
+        } catch (Exception e) {
+            writer.fail();
+            throw new HyracksDataException(e);
+        } finally {
+            writer.close();
+        }
+    }
+
+    private void merge(IFrameWriter mergeResultWriter, IFrameReader[] runCursors) throws HyracksDataException {
+        RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields, comparators,
+                recordDesc);
+        merger.open();
+        try {
+            while (merger.nextFrame(outFrame)) {
+                FrameUtils.flushFrame(outFrame, mergeResultWriter);
+            }
+        } finally {
+            merger.close();
+        }
+    }
+
+    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 IllegalStateException();
+                        }
+                        totalCount++;
+                    }
+                }
+                if (outFrameAppender.getTupleCount() > 0) {
+                    FrameUtils.flushFrame(outFrame, writer);
+                    outFrameAppender.reset(outFrame, true);
+                }
+                return;
+            }
+            // 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);
+                }
+            }
+
+        } catch (Exception e) {
+            writer.fail();
+            throw new HyracksDataException(e);
+        } finally {
+            writer.close();
+        }
+    }
+
+    // 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, 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();
+            }
+
+            if (outputLimit > 0 && finalPass && (currentSize >= outputLimit)) {
+                runs.clear();
+                return;
+            }
+
+            runs.subList(0, inFrames.size()).clear();
+            if (!finalPass) {
+                runs.add(0, ((RunFileWriter) writer).createReader());
+            }
+        } finally {
+            if (!finalPass) {
+                writer.close();
+            }
+        }
+    }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
new file mode 100644
index 0000000..680b98e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
@@ -0,0 +1,245 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.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;
+
+public class FrameSorter {
+    private final IHyracksTaskContext ctx;
+    private final int[] sortFields;
+    private final INormalizedKeyComputer nkc;
+    private final IBinaryComparator[] comparators;
+    private final List<ByteBuffer> buffers;
+
+    private final FrameTupleAccessor fta1;
+    private final FrameTupleAccessor fta2;
+
+    private final FrameTupleAppender appender;
+
+    private final ByteBuffer outFrame;
+
+    private int dataFrameCount;
+    private int[] tPointers;
+    private int tupleCount;
+
+    public FrameSorter(IHyracksTaskContext ctx, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor) {
+        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;
+    }
+
+    public void reset() {
+        dataFrameCount = 0;
+        tupleCount = 0;
+    }
+
+    public int getFrameCount() {
+        return dataFrameCount;
+    }
+
+    public void insertFrame(ByteBuffer buffer) {
+        ByteBuffer copyFrame;
+        if (dataFrameCount == buffers.size()) {
+            copyFrame = ctx.allocateFrame();
+            buffers.add(copyFrame);
+        } else {
+            copyFrame = buffers.get(dataFrameCount);
+        }
+        FrameUtils.copy(buffer, copyFrame);
+        ++dataFrameCount;
+    }
+
+    public void sortFrames() {
+        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 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 IllegalStateException();
+                }
+            }
+        }
+        if (appender.getTupleCount() > 0) {
+            FrameUtils.flushFrame(outFrame, writer);
+        }
+    }
+
+    private void sort(int[] tPointers, int offset, int length) {
+        int m = offset + (length >> 1);
+        int mi = tPointers[m * 4];
+        int mj = tPointers[m * 4 + 1];
+        int mv = tPointers[m * 4 + 3];
+
+        int a = offset;
+        int b = a;
+        int c = offset + length - 1;
+        int d = c;
+        while (true) {
+            while (b <= c) {
+                int cmp = compare(tPointers, b, mi, mj, mv);
+                if (cmp > 0) {
+                    break;
+                }
+                if (cmp == 0) {
+                    swap(tPointers, a++, b);
+                }
+                ++b;
+            }
+            while (c >= b) {
+                int cmp = compare(tPointers, c, mi, mj, mv);
+                if (cmp < 0) {
+                    break;
+                }
+                if (cmp == 0) {
+                    swap(tPointers, c, d--);
+                }
+                --c;
+            }
+            if (b > c)
+                break;
+            swap(tPointers, b++, c--);
+        }
+
+        int s;
+        int n = offset + length;
+        s = Math.min(a - offset, b - a);
+        vecswap(tPointers, offset, b - s, s);
+        s = Math.min(d - c, n - d - 1);
+        vecswap(tPointers, b, n - s, s);
+
+        if ((s = b - a) > 1) {
+            sort(tPointers, offset, s);
+        }
+        if ((s = d - c) > 1) {
+            sort(tPointers, n - s, s);
+        }
+    }
+
+    private void swap(int x[], int a, int b) {
+        for (int i = 0; i < 4; ++i) {
+            int t = x[a * 4 + i];
+            x[a * 4 + i] = x[b * 4 + i];
+            x[b * 4 + i] = t;
+        }
+    }
+
+    private void vecswap(int x[], int a, int b, int n) {
+        for (int i = 0; i < n; i++, a++, b++) {
+            swap(x, a, b);
+        }
+    }
+
+    private int compare(int[] tPointers, int tp1, int tp2i, int tp2j, int tp2v) {
+        int i1 = tPointers[tp1 * 4];
+        int j1 = tPointers[tp1 * 4 + 1];
+        int v1 = tPointers[tp1 * 4 + 3];
+        if (v1 != tp2v) {
+            return ((((long) v1) & 0xffffffffL) < (((long) tp2v) & 0xffffffffL)) ? -1 : 1;
+        }
+        int i2 = tp2i;
+        int j2 = tp2j;
+        ByteBuffer buf1 = buffers.get(i1);
+        ByteBuffer buf2 = buffers.get(i2);
+        byte[] b1 = buf1.array();
+        byte[] b2 = buf2.array();
+        fta1.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 l1 = f1End - f1Start;
+            int f2Start = fIdx == 0 ? 0 : buf2.getInt(j2 + (fIdx - 1) * 4);
+            int f2End = buf2.getInt(j2 + fIdx * 4);
+            int s2 = j2 + fta2.getFieldSlotsLength() + f2Start;
+            int l2 = f2End - f2Start;
+            int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+            if (c != 0) {
+                return c;
+            }
+        }
+        return 0;
+    }
+
+    public void close() {
+        this.buffers.clear();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IMemoryManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IMemoryManager.java
new file mode 100644
index 0000000..418b4ad
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IMemoryManager.java
@@ -0,0 +1,74 @@
+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
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
new file mode 100644
index 0000000..69cf8b1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IRunGenerator.java
@@ -0,0 +1,18 @@
+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;
+
+/**
+ * @author pouria
+ *         Interface for the Run Generator
+ */
+public interface IRunGenerator extends IFrameWriter {
+
+    /**
+     * @return the list of generated (sorted) runs
+     */
+    public List<IFrameReader> getRuns();
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..a7d6c08
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISelectionTree.java
@@ -0,0 +1,70 @@
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+/**
+ * @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
+     */
+    void insert(int[] element);
+
+    /**
+     * Removes and returns the smallest element in the tree
+     * 
+     * @param result
+     *            is the array that will eventually contain minimum entry
+     *            pointer
+     */
+    void getMin(int[] result);
+
+    /**
+     * Removes and returns the largest element in the tree
+     * 
+     * @param result
+     *            is the array that will eventually contain maximum entry
+     *            pointer
+     */
+    void getMax(int[] result);
+
+    /**
+     * @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
+     */
+    void peekMax(int[] result);
+
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..3b3c28d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -0,0 +1,167 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+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.IBinaryComparatorFactory;
+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 class InMemorySortOperatorDescriptor extends AbstractOperatorDescriptor {
+    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 INormalizedKeyComputerFactory firstKeyNormalizerFactory;
+    private IBinaryComparatorFactory[] comparatorFactories;
+
+    public InMemorySortOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] sortFields,
+            IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
+        this(spec, sortFields, null, comparatorFactories, recordDescriptor);
+    }
+
+    public InMemorySortOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] sortFields,
+            INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.sortFields = sortFields;
+        this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
+        this.comparatorFactories = comparatorFactories;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @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 FrameSorter 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 {
+        }
+    }
+
+    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 SortTaskState state;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition));
+                    state.frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory,
+                            comparatorFactories, recordDescriptors[0]);
+                    state.frameSorter.reset();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    state.frameSorter.insertFrame(buffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    state.frameSorter.sortFrames();
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                }
+            };
+            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,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
+                @Override
+                public void initialize() throws HyracksDataException {
+                    writer.open();
+                    try {
+                        SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(
+                                getOperatorId(), SORT_ACTIVITY_ID), partition));
+                        state.frameSorter.flushFrames(writer);
+                    } catch (Exception e) {
+                        writer.fail();
+                        throw new HyracksDataException(e);
+                    } finally {
+                        writer.close();
+                    }
+                }
+            };
+            return op;
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..d9c16d8
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
@@ -0,0 +1,215 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.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.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();
+                    }
+
+                    int necessaryFrames = Math.min(runs.size() + 2, memSize);
+                    ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, outputLimit, runs, sortFields,
+                            comparators, recordDescriptors[0], necessaryFrames, writer);
+
+                    merger.processWithReplacementSelection();
+
+                }
+            };
+            return op;
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..f01d886
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGenerator.java
@@ -0,0 +1,301 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.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) { // 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
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
new file mode 100644
index 0000000..25442b3
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGeneratorWithLimit.java
@@ -0,0 +1,454 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.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) { // 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) {
+        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
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
new file mode 100644
index 0000000..29c28a1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
@@ -0,0 +1,169 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.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.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 int[] sortFields;
+    private final IBinaryComparator[] comparators;
+    private final RecordDescriptor recordDesc;
+    private final FrameTupleAppender outFrameAppender;
+    private ReferencedPriorityQueue topTuples;
+    private int[] tupleIndexes;
+    private FrameTupleAccessor[] tupleAccessors;
+
+    public RunMergingFrameReader(IHyracksTaskContext ctx, IFrameReader[] runCursors, List<ByteBuffer> inFrames,
+            int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc) {
+        this.ctx = ctx;
+        this.runCursors = runCursors;
+        this.inFrames = inFrames;
+        this.sortFields = sortFields;
+        this.comparators = comparators;
+        this.recordDesc = recordDesc;
+        outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        tupleAccessors = new FrameTupleAccessor[runCursors.length];
+        Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
+        topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, runCursors.length, comparator);
+        tupleIndexes = new int[runCursors.length];
+        for (int i = 0; i < runCursors.length; 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);
+            } else {
+                closeRun(runIndex, runCursors, tupleAccessors);
+                topTuples.pop();
+            }
+        }
+    }
+
+    @Override
+    public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        outFrameAppender.reset(buffer, true);
+        while (!topTuples.areRunsExhausted()) {
+            ReferenceEntry top = topTuples.peek();
+            int runIndex = top.getRunid();
+            FrameTupleAccessor fta = top.getAccessor();
+            int tupleIndex = top.getTupleIndex();
+
+            if (!outFrameAppender.append(fta, tupleIndex)) {
+                return true;
+            }
+
+            ++tupleIndexes[runIndex];
+            setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+        }
+
+        if (outFrameAppender.getTupleCount() > 0) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        for (int i = 0; i < runCursors.length; ++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);
+        if (exists) {
+            topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
+        } else {
+            topTuples.pop();
+            closeRun(runIndex, runCursors, tupleAccessors);
+        }
+    }
+
+    private boolean hasNextTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
+            FrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
+        if (tupleAccessors[runIndex] == null || runCursors[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)) {
+                tupleIndexes[runIndex] = 0;
+                return hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+            } else {
+                return false;
+            }
+        } else {
+            return true;
+        }
+    }
+
+    private void closeRun(int index, IFrameReader[] runCursors, IFrameTupleAccessor[] tupleAccessors)
+            throws HyracksDataException {
+        if (runCursors[index] != null) {
+            runCursors[index].close();
+            runCursors[index] = null;
+            tupleAccessors[index] = null;
+        }
+    }
+
+    private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
+        return new Comparator<ReferenceEntry>() {
+            public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
+                FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
+                FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
+                int j1 = tp1.getTupleIndex();
+                int j2 = tp2.getTupleIndex();
+                byte[] b1 = fta1.getBuffer().array();
+                byte[] b2 = fta2.getBuffer().array();
+                for (int f = 0; f < sortFields.length; ++f) {
+                    int fIdx = sortFields[f];
+                    int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
+                            + fta1.getFieldStartOffset(j1, fIdx);
+                    int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
+                    int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
+                            + fta2.getFieldStartOffset(j2, fIdx);
+                    int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
+                    int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+                    if (c != 0) {
+                        return c;
+                    }
+                }
+                return 0;
+            }
+        };
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..aaa9049
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Slot.java
@@ -0,0 +1,67 @@
+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
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
new file mode 100644
index 0000000..a221118
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinHeap.java
@@ -0,0 +1,277 @@
+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.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) {
+        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) {
+        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) {
+        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) {
+        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) {
+        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) {
+        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) {
+        // 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) {
+        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) { // 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
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
new file mode 100644
index 0000000..dcaf781
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinMaxHeap.java
@@ -0,0 +1,432 @@
+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.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) {
+        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) {
+        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) {
+        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) {
+        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) {
+        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) {
+        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) {
+        int gp = getGrandParent(ix);
+        if (gp != NOT_EXIST && compare(gp, ix) < 0) {
+            swap(ix, gp);
+            bubbleUpMax(gp);
+        }
+    }
+
+    private void bubbleUpMin(int ix) {
+        int gp = getGrandParent(ix);
+        if (gp != NOT_EXIST && compare(ix, gp) < 0) {
+            swap(ix, gp);
+            bubbleUpMin(gp);
+        }
+    }
+
+    private void trickleDown(int ix) {
+        if (isAtMinLevel(ix)) {
+            trickleDownMin(ix);
+        } else {
+            trickleDownMax(ix);
+        }
+    }
+
+    private void trickleDownMax(int ix) {
+        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) {
+        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) {
+        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) {
+        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) {
+        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) {
+        // 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) {
+        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
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java
new file mode 100644
index 0000000..7dc0b17
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/ISerializableTable.java
@@ -0,0 +1,16 @@
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface ISerializableTable {
+
+    public void insert(int entry, TuplePointer tuplePointer);
+
+    public void getTuplePointer(int entry, int offset, TuplePointer tuplePointer);
+
+    public int getFrameCount();
+
+    public int getTupleCount();
+
+    public void reset();
+
+    public void close();
+}
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
new file mode 100644
index 0000000..0acafee
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
@@ -0,0 +1,267 @@
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * An entry in the table is: #elements, #no-empty elements; fIndex, tIndex;
+ * fIndex, tIndex; .... <fIndex, tIndex> forms a tuple pointer
+ */
+public class SerializableHashTable implements ISerializableTable {
+
+    private static final int INT_SIZE = 4;
+    private static final int INIT_ENTRY_SIZE = 4;
+
+    private IntSerDeBuffer[] headers;
+    private List<IntSerDeBuffer> contents = new ArrayList<IntSerDeBuffer>();
+    private List<Integer> frameCurrentIndex = new ArrayList<Integer>();
+    private final IHyracksTaskContext ctx;
+    private int frameCapacity = 0;
+    private int currentLargestFrameIndex = 0;
+    private int tupleCount = 0;
+    private int headerFrameCount = 0;
+    private TuplePointer tempTuplePointer = new TuplePointer();
+
+    public SerializableHashTable(int tableSize, final IHyracksTaskContext ctx) {
+        this.ctx = ctx;
+        int frameSize = ctx.getFrameSize();
+
+        int residual = tableSize * INT_SIZE * 2 % frameSize == 0 ? 0 : 1;
+        int headerSize = tableSize * INT_SIZE * 2 / frameSize + residual;
+        headers = new IntSerDeBuffer[headerSize];
+
+        IntSerDeBuffer frame = new IntSerDeBuffer(ctx.allocateFrame().array());
+        contents.add(frame);
+        frameCurrentIndex.add(0);
+        frameCapacity = frame.capacity();
+    }
+
+    @Override
+    public void insert(int entry, TuplePointer pointer) {
+        int hFrameIndex = getHeaderFrameIndex(entry);
+        int headerOffset = getHeaderFrameOffset(entry);
+        IntSerDeBuffer header = headers[hFrameIndex];
+        if (header == null) {
+            header = new IntSerDeBuffer(ctx.allocateFrame().array());
+            headers[hFrameIndex] = header;
+            resetFrame(header);
+            headerFrameCount++;
+        }
+        int frameIndex = header.getInt(headerOffset);
+        int offsetIndex = header.getInt(headerOffset + 1);
+        if (frameIndex < 0) {
+            // insert first tuple into the entry
+            insertNewEntry(header, headerOffset, INIT_ENTRY_SIZE, pointer);
+        } else {
+            // insert non-first tuple into the entry
+            insertNonFirstTuple(header, headerOffset, frameIndex, offsetIndex, pointer);
+        }
+        tupleCount++;
+    }
+
+    @Override
+    public void getTuplePointer(int entry, int offset, TuplePointer dataPointer) {
+        int hFrameIndex = getHeaderFrameIndex(entry);
+        int headerOffset = getHeaderFrameOffset(entry);
+        IntSerDeBuffer header = headers[hFrameIndex];
+        if (header == null) {
+            dataPointer.frameIndex = -1;
+            dataPointer.tupleIndex = -1;
+            return;
+        }
+        int frameIndex = header.getInt(headerOffset);
+        int offsetIndex = header.getInt(headerOffset + 1);
+        if (frameIndex < 0) {
+            dataPointer.frameIndex = -1;
+            dataPointer.tupleIndex = -1;
+            return;
+        }
+        IntSerDeBuffer frame = contents.get(frameIndex);
+        int entryUsedItems = frame.getInt(offsetIndex + 1);
+        if (offset > entryUsedItems - 1) {
+            dataPointer.frameIndex = -1;
+            dataPointer.tupleIndex = -1;
+            return;
+        }
+        int startIndex = offsetIndex + 2 + offset * 2;
+        while (startIndex >= frameCapacity) {
+            ++frameIndex;
+            startIndex -= frameCapacity;
+        }
+        frame = contents.get(frameIndex);
+        dataPointer.frameIndex = frame.getInt(startIndex);
+        dataPointer.tupleIndex = frame.getInt(startIndex + 1);
+    }
+
+    @Override
+    public void reset() {
+        for (IntSerDeBuffer frame : headers)
+            if (frame != null)
+                resetFrame(frame);
+
+        frameCurrentIndex.clear();
+        for (int i = 0; i < contents.size(); i++) {
+            frameCurrentIndex.add(0);
+        }
+
+        currentLargestFrameIndex = 0;
+        tupleCount = 0;
+    }
+
+    @Override
+    public int getFrameCount() {
+        return headerFrameCount + contents.size();
+    }
+
+    public int getTupleCount() {
+        return tupleCount;
+    }
+
+    @Override
+    public void close() {
+        for (int i = 0; i < headers.length; i++)
+            headers[i] = null;
+        contents.clear();
+        frameCurrentIndex.clear();
+        tupleCount = 0;
+        currentLargestFrameIndex = 0;
+    }
+
+    private void insertNewEntry(IntSerDeBuffer header, int headerOffset, int entryCapacity, TuplePointer pointer) {
+        IntSerDeBuffer lastFrame = contents.get(currentLargestFrameIndex);
+        int lastIndex = frameCurrentIndex.get(currentLargestFrameIndex);
+        int requiredIntCapacity = entryCapacity * 2;
+        int startFrameIndex = currentLargestFrameIndex;
+
+        if (lastIndex + requiredIntCapacity >= frameCapacity) {
+            IntSerDeBuffer newFrame;
+            startFrameIndex++;
+            do {
+                if (currentLargestFrameIndex >= contents.size() - 1) {
+                    newFrame = new IntSerDeBuffer(ctx.allocateFrame().array());
+                    currentLargestFrameIndex++;
+                    contents.add(newFrame);
+                    frameCurrentIndex.add(0);
+                } else {
+                    currentLargestFrameIndex++;
+                    frameCurrentIndex.set(currentLargestFrameIndex, 0);
+                }
+                requiredIntCapacity -= frameCapacity;
+            } while (requiredIntCapacity > 0);
+            lastIndex = 0;
+            lastFrame = contents.get(startFrameIndex);
+        }
+
+        // set header
+        header.writeInt(headerOffset, startFrameIndex);
+        header.writeInt(headerOffset + 1, lastIndex);
+
+        // set the entry
+        lastFrame.writeInt(lastIndex, entryCapacity - 1);
+        lastFrame.writeInt(lastIndex + 1, 1);
+        lastFrame.writeInt(lastIndex + 2, pointer.frameIndex);
+        lastFrame.writeInt(lastIndex + 3, pointer.tupleIndex);
+        int newLastIndex = lastIndex + entryCapacity * 2;
+        newLastIndex = newLastIndex < frameCapacity ? newLastIndex : frameCapacity - 1;
+        frameCurrentIndex.set(startFrameIndex, newLastIndex);
+
+        requiredIntCapacity = entryCapacity * 2 - (frameCapacity - lastIndex);
+        while (requiredIntCapacity > 0) {
+            startFrameIndex++;
+            requiredIntCapacity -= frameCapacity;
+            newLastIndex = requiredIntCapacity < 0 ? requiredIntCapacity + frameCapacity : frameCapacity - 1;
+            frameCurrentIndex.set(startFrameIndex, newLastIndex);
+        }
+    }
+
+    private void insertNonFirstTuple(IntSerDeBuffer header, int headerOffset, int frameIndex, int offsetIndex,
+            TuplePointer pointer) {
+        IntSerDeBuffer frame = contents.get(frameIndex);
+        int entryItems = frame.getInt(offsetIndex);
+        int entryUsedItems = frame.getInt(offsetIndex + 1);
+
+        if (entryUsedItems < entryItems) {
+            frame.writeInt(offsetIndex + 1, entryUsedItems + 1);
+            int startIndex = offsetIndex + 2 + entryUsedItems * 2;
+            while (startIndex >= frameCapacity) {
+                ++frameIndex;
+                startIndex -= frameCapacity;
+            }
+            frame = contents.get(frameIndex);
+            frame.writeInt(startIndex, pointer.frameIndex);
+            frame.writeInt(startIndex + 1, pointer.tupleIndex);
+        } else {
+            int capacity = (entryItems + 1) * 2;
+            header.writeInt(headerOffset, -1);
+            header.writeInt(headerOffset + 1, -1);
+            int fIndex = frame.getInt(offsetIndex + 2);
+            int tIndex = frame.getInt(offsetIndex + 3);
+            tempTuplePointer.frameIndex = fIndex;
+            tempTuplePointer.tupleIndex = tIndex;
+            this.insertNewEntry(header, headerOffset, capacity, tempTuplePointer);
+            int newFrameIndex = header.getInt(headerOffset);
+            int newTupleIndex = header.getInt(headerOffset + 1);
+
+            for (int i = 1; i < entryUsedItems; i++) {
+                int startIndex = offsetIndex + 2 + i * 2;
+                int startFrameIndex = frameIndex;
+                while (startIndex >= frameCapacity) {
+                    ++startFrameIndex;
+                    startIndex -= frameCapacity;
+                }
+                frame = contents.get(startFrameIndex);
+                fIndex = frame.getInt(startIndex);
+                tIndex = frame.getInt(startIndex + 1);
+                tempTuplePointer.frameIndex = fIndex;
+                tempTuplePointer.tupleIndex = tIndex;
+                insertNonFirstTuple(header, headerOffset, newFrameIndex, newTupleIndex, tempTuplePointer);
+            }
+            insertNonFirstTuple(header, headerOffset, newFrameIndex, newTupleIndex, pointer);
+        }
+    }
+
+    private void resetFrame(IntSerDeBuffer frame) {
+        for (int i = 0; i < frameCapacity; i++)
+            frame.writeInt(i, -1);
+    }
+
+    private int getHeaderFrameIndex(int entry) {
+        int frameIndex = entry * 2 / frameCapacity;
+        return frameIndex;
+    }
+
+    private int getHeaderFrameOffset(int entry) {
+        int offset = entry * 2 % frameCapacity;
+        return offset;
+    }
+
+}
+
+class IntSerDeBuffer {
+
+    private byte[] bytes;
+
+    public IntSerDeBuffer(byte[] data) {
+        this.bytes = data;
+    }
+
+    public int getInt(int pos) {
+        int offset = pos * 4;
+        return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
+                + ((bytes[offset + 3] & 0xff) << 0);
+    }
+
+    public void writeInt(int pos, int value) {
+        int offset = pos * 4;
+        bytes[offset++] = (byte) (value >> 24);
+        bytes[offset++] = (byte) (value >> 16);
+        bytes[offset++] = (byte) (value >> 8);
+        bytes[offset++] = (byte) (value);
+    }
+
+    public int capacity() {
+        return bytes.length / 4;
+    }
+}
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
new file mode 100644
index 0000000..6618fb1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/TuplePointer.java
@@ -0,0 +1,6 @@
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public class TuplePointer {
+    public int frameIndex;
+    public int tupleIndex;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
new file mode 100644
index 0000000..7ac9f58
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.union;
+
+import java.nio.ByteBuffer;
+
+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.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.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputOperatorNodePushable;
+
+public class UnionAllOperatorDescriptor extends AbstractOperatorDescriptor {
+    public UnionAllOperatorDescriptor(IOperatorDescriptorRegistry spec, int nInputs, RecordDescriptor recordDescriptor) {
+        super(spec, nInputs, 1);
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        UnionActivityNode uba = new UnionActivityNode(new ActivityId(getOperatorId(), 0));
+        builder.addActivity(this, uba);
+        for (int i = 0; i < inputArity; ++i) {
+            builder.addSourceEdge(i, uba, i);
+        }
+        builder.addTargetEdge(0, uba, 0);
+    }
+
+    private class UnionActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public UnionActivityNode(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+                throws HyracksDataException {
+            RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            return new UnionOperator(ctx, inRecordDesc);
+        }
+    }
+
+    private class UnionOperator extends AbstractUnaryOutputOperatorNodePushable {
+        private int nOpened;
+
+        private int nClosed;
+
+        private boolean failed;
+
+        public UnionOperator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc) {
+            nOpened = 0;
+            nClosed = 0;
+        }
+
+        @Override
+        public int getInputArity() {
+            return inputArity;
+        }
+
+        @Override
+        public IFrameWriter getInputFrameWriter(int index) {
+            return new IFrameWriter() {
+                @Override
+                public void open() throws HyracksDataException {
+                    synchronized (UnionOperator.this) {
+                        if (++nOpened == 1) {
+                            writer.open();
+                        }
+                    }
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    synchronized (UnionOperator.this) {
+                        writer.nextFrame(buffer);
+                    }
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    synchronized (UnionOperator.this) {
+                        if (failed) {
+                            writer.fail();
+                        }
+                        failed = true;
+                    }
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    synchronized (UnionOperator.this) {
+                        if (++nClosed == inputArity) {
+                            writer.close();
+                        }
+                    }
+                }
+            };
+        }
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..3345dd4
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.util;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.SerializingDataWriter;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
+
+public final class DeserializedOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
+    private final IHyracksTaskContext ctx;
+
+    private final IOpenableDataWriterOperator delegate;
+
+    private final FrameDeserializer deserializer;
+
+    public DeserializedOperatorNodePushable(IHyracksTaskContext ctx, IOpenableDataWriterOperator delegate,
+            RecordDescriptor inRecordDesc) {
+        this.ctx = ctx;
+        this.delegate = delegate;
+        deserializer = inRecordDesc == null ? null : new FrameDeserializer(ctx.getFrameSize(), inRecordDesc);
+    }
+
+    @Override
+    public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+        delegate.setDataWriter(index, new SerializingDataWriter(ctx, recordDesc, writer));
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        delegate.close();
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        deserializer.reset(buffer);
+        while (!deserializer.done()) {
+            delegate.writeData(deserializer.deserializeRecord());
+        }
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        delegate.open();
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        delegate.fail();
+    }
+
+    @Override
+    public String getDisplayName() {
+        return "Deserialized(" + delegate + ")";
+    }
+}
\ No newline at end of file
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
new file mode 100644
index 0000000..eafe2d7
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.util;
+
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class ReferenceEntry {
+    private final int runid;
+    private FrameTupleAccessor acccessor;
+    private int tupleIndex;
+
+    public ReferenceEntry(int runid, FrameTupleAccessor fta, int tupleIndex) {
+        super();
+        this.runid = runid;
+        this.acccessor = fta;
+        this.tupleIndex = tupleIndex;
+    }
+
+    public int getRunid() {
+        return runid;
+    }
+
+    public FrameTupleAccessor getAccessor() {
+        return acccessor;
+    }
+
+    public void setAccessor(FrameTupleAccessor fta) {
+        this.acccessor = fta;
+    }
+
+    public int getTupleIndex() {
+        return tupleIndex;
+    }
+
+    public void setTupleIndex(int tupleIndex) {
+        this.tupleIndex = tupleIndex;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
new file mode 100644
index 0000000..b209a4a
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
@@ -0,0 +1,143 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.util;
+
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.Comparator;
+
+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;
+    private int nItems;
+
+    private final Comparator<ReferenceEntry> comparator;
+
+    public ReferencedPriorityQueue(int frameSize, RecordDescriptor recordDescriptor, int initSize,
+            Comparator<ReferenceEntry> comparator) {
+        this.frameSize = frameSize;
+        this.recordDescriptor = recordDescriptor;
+        if (initSize < 1)
+            throw new IllegalArgumentException();
+        this.comparator = comparator;
+        nItems = initSize;
+        size = (initSize + 1) & 0xfffffffe;
+        entries = new ReferenceEntry[size];
+        runAvail = new BitSet(size);
+        runAvail.set(0, initSize, true);
+        for (int i = 0; i < size; i++) {
+            entries[i] = new ReferenceEntry(i, null, -1);
+        }
+    }
+
+    /**
+     * Retrieve the top entry without removing it
+     * 
+     * @return the top entry
+     */
+    public ReferenceEntry peek() {
+        return entries[0];
+    }
+
+    /**
+     * compare the new entry with entries within the queue, to find a spot for
+     * this new entry
+     * 
+     * @param entry
+     * @return runid of this entry
+     * @throws IOException
+     */
+    public int popAndReplace(FrameTupleAccessor fta, int tIndex) {
+        ReferenceEntry entry = entries[0];
+        if (entry.getAccessor() == null) {
+            entry.setAccessor(new FrameTupleAccessor(frameSize, recordDescriptor));
+        }
+        entry.getAccessor().reset(fta.getBuffer());
+        entry.setTupleIndex(tIndex);
+
+        add(entry);
+        return entry.getRunid();
+    }
+
+    /**
+     * Push entry into priority queue
+     * 
+     * @param e
+     *            the new Entry
+     */
+    private void add(ReferenceEntry e) {
+        ReferenceEntry min = entries[0];
+        int slot = (size >> 1) + (min.getRunid() >> 1);
+
+        ReferenceEntry curr = e;
+        while (!runAvail.isEmpty() && slot > 0) {
+            int c = 0;
+            if (!runAvail.get(entries[slot].getRunid())) {
+                // run of entries[slot] is exhausted, i.e. not available, curr
+                // wins
+                c = 1;
+            } else if (entries[slot].getAccessor() != null /*
+                                                            * entries[slot] is
+                                                            * not MIN value
+                                                            */
+                    && runAvail.get(curr.getRunid() /* curr run is available */)) {
+
+                if (curr.getAccessor() != null) {
+                    c = comparator.compare(entries[slot], curr);
+                } else {
+                    // curr is MIN value, wins
+                    c = 1;
+                }
+            }
+
+            if (c <= 0) { // curr lost
+                // entries[slot] swaps up
+                ReferenceEntry tmp = entries[slot];
+                entries[slot] = curr;
+                curr = tmp;// winner to pass up
+            }// else curr wins
+            slot >>= 1;
+        }
+        // set new entries[0]
+        entries[0] = curr;
+    }
+
+    /**
+     * Pop is called only when a run is exhausted
+     * 
+     * @return
+     */
+    public ReferenceEntry pop() {
+        ReferenceEntry min = entries[0];
+        runAvail.clear(min.getRunid());
+        add(min);
+        nItems--;
+        return min;
+    }
+
+    public boolean areRunsExhausted() {
+        return runAvail.isEmpty();
+    }
+
+    public int size() {
+        return nItems;
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SelectionTree.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SelectionTree.java
new file mode 100644
index 0000000..2a288cc
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/SelectionTree.java
@@ -0,0 +1,107 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.util;
+
+import java.util.BitSet;
+
+public class SelectionTree {
+    private int size;
+
+    private Entry[] entries;
+
+    private int[] losers;
+
+    private BitSet available;
+
+    public SelectionTree(Entry[] e) {
+        size = (e.length + 1) & 0xfffffffe;
+        available = new BitSet(size);
+        available.set(0, e.length, true);
+        losers = new int[size];
+        entries = e;
+        for (int i = 0; i < size; ++i) {
+            losers[i] = -1;
+        }
+        for (int i = 0; i < size; ++i) {
+            int slot = (size + i) >> 1;
+
+            if (i < entries.length) {
+                available.set(i, entries[i].advance());
+            }
+            int currIdx = i;
+            while (slot > 0) {
+                int cmp = 0;
+                if (losers[slot] < 0 || currIdx < 0) {
+                    cmp = losers[slot] < 0 ? -1 : 1;
+                } else if (!available.get(losers[slot])) {
+                    cmp = 1;
+                } else if (available.get(currIdx)) {
+                    if (currIdx <= i) {
+                        cmp = entries[losers[slot]].compareTo(entries[currIdx]);
+                    } else {
+                        cmp = 1;
+                    }
+                }
+
+                if (cmp <= 0) {
+                    int tmp = losers[slot];
+                    losers[slot] = currIdx;
+                    currIdx = tmp;
+                }
+                slot >>= 1;
+            }
+            losers[0] = currIdx;
+        }
+    }
+
+    public Entry peek() {
+        if (entries.length == 0) {
+            return null;
+        }
+        return entries[losers[0]];
+    }
+
+    public void pop() {
+        int winner = losers[0];
+        int slot = (size + winner) >> 1;
+
+        boolean avail = entries[winner].advance();
+        if (!avail) {
+            entries[winner] = null;
+        }
+        available.set(winner, avail);
+        int currIdx = winner;
+        while (!available.isEmpty() && slot > 0) {
+            int cmp = 0;
+            if (!available.get(losers[slot])) {
+                cmp = 1;
+            } else if (available.get(currIdx)) {
+                cmp = entries[losers[slot]].compareTo(entries[currIdx]);
+            }
+
+            if (cmp <= 0) {
+                int tmp = losers[slot];
+                losers[slot] = currIdx;
+                currIdx = tmp;
+            }
+            slot >>= 1;
+        }
+        losers[0] = currIdx;
+    }
+
+    public interface Entry extends Comparable<Entry> {
+        public abstract boolean advance();
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/StringSerializationUtils.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/StringSerializationUtils.java
new file mode 100644
index 0000000..1126c1f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/StringSerializationUtils.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.util;
+
+import java.util.Arrays;
+
+public class StringSerializationUtils {
+    public static String toString(Object object) {
+        if (object instanceof Object[]) {
+            return Arrays.deepToString((Object[]) object);
+        } else {
+            return String.valueOf(object);
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/test/util/SelectionTreeTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/test/util/SelectionTreeTest.java
new file mode 100644
index 0000000..8f467a0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/test/util/SelectionTreeTest.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.test.util;
+
+import junit.framework.Assert;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.dataflow.std.util.SelectionTree;
+import edu.uci.ics.hyracks.dataflow.std.util.SelectionTree.Entry;
+
+public class SelectionTreeTest {
+    @Test
+    public void sortMergeTest() {
+        SelectionTree.Entry[] entries = new SelectionTree.Entry[5];
+        for (int i = 0; i < entries.length; ++i) {
+            entries[i] = new MergeEntry(0, i * entries.length + i, entries.length);
+        }
+        SelectionTree tree = new SelectionTree(entries);
+        SelectionTree.Entry e;
+        int last = Integer.MIN_VALUE;
+        while ((e = tree.peek()) != null) {
+            MergeEntry me = (MergeEntry) e;
+            if (me.i < last) {
+                Assert.fail();
+            }
+            last = me.i;
+            tree.pop();
+        }
+    }
+
+    private static class MergeEntry implements SelectionTree.Entry {
+        private int i;
+        private int max;
+        private int step;
+
+        public MergeEntry(int min, int max, int step) {
+            this.max = max;
+            this.step = step;
+            i = min - step;
+        }
+
+        @Override
+        public int compareTo(Entry o) {
+            return i < ((MergeEntry) o).i ? -1 : (i == ((MergeEntry) o).i ? 0 : 1);
+        }
+
+        @Override
+        public boolean advance() {
+            if (i > max) {
+                return false;
+            }
+            i += step;
+            return true;
+        }
+    }
+}
\ No newline at end of file